You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by sr...@apache.org on 2017/05/14 12:10:24 UTC

[01/14] flink git commit: [FLINK-6461] Deprecate web config defaults in ConfigConstants

Repository: flink
Updated Branches:
  refs/heads/release-1.3 670b23e97 -> 471263cfe


[FLINK-6461] Deprecate web config defaults in ConfigConstants

This closes #3831.


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

Branch: refs/heads/release-1.3
Commit: 9708550a40127f833728bcc85847035c3b5fbad8
Parents: 60deaef
Author: zentol <ch...@apache.org>
Authored: Fri May 5 12:39:55 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:06:28 2017 +0200

----------------------------------------------------------------------
 .../client/program/StandaloneClusterClient.java |  5 +-
 .../flink/configuration/ConfigConstants.java    | 65 +++++++++++++++++---
 .../flink/configuration/JobManagerOptions.java  |  7 +++
 .../flink/api/java/ExecutionEnvironment.java    |  4 --
 .../runtime/webmonitor/WebMonitorConfig.java    | 36 +----------
 .../runtime/webmonitor/WebRuntimeMonitor.java   | 32 +++-------
 .../webmonitor/WebRuntimeMonitorITCase.java     | 13 ++--
 .../executiongraph/ExecutionGraphBuilder.java   |  5 +-
 .../runtime/webmonitor/WebMonitorUtils.java     |  6 +-
 .../flink/runtime/jobmanager/JobManager.scala   |  3 +-
 .../environment/StreamExecutionEnvironment.java |  4 --
 .../apache/flink/test/util/TestBaseUtils.java   |  5 +-
 .../flink/test/web/WebFrontendITCase.java       |  5 +-
 .../flink/yarn/YarnApplicationMasterRunner.java |  3 +-
 14 files changed, 98 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
index fd179c0..7517504 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -18,8 +18,8 @@
 package org.apache.flink.client.program;
 
 import org.apache.flink.api.common.JobSubmissionResult;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
@@ -53,8 +53,7 @@ public class StandaloneClusterClient extends ClusterClient {
 	@Override
 	public String getWebInterfaceURL() {
 		String host = this.getJobManagerAddress().getHostString();
-		int port = getFlinkConfiguration().getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY,
-			ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT);
+		int port = getFlinkConfiguration().getInteger(JobManagerOptions.WEB_PORT);
 		return "http://" +  host + ":" + port;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 92e6b5d..c3704be 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -1338,36 +1338,81 @@ public final class ConfigConstants {
 		key("jobmanager.web.address")
 			.noDefaultValue();
 
-	/** The config key for the port of the JobManager web frontend.
-	 * Setting this value to {@code -1} disables the web frontend. */
+	/**
+	 * The config key for the port of the JobManager web frontend.
+	 * Setting this value to {@code -1} disables the web frontend.
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_PORT} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT = 8081;
 
-	/** Default value to override SSL support for the JobManager web UI */
+	/**
+	 * Default value to override SSL support for the JobManager web UI
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_SSL_ENABLED} instead
+	 */
+	@Deprecated
 	public static final boolean DEFAULT_JOB_MANAGER_WEB_SSL_ENABLED = true;
 
-	/** The default number of archived jobs for the jobmanager */
+	/**
+	 * The default number of archived jobs for the jobmanager
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_ARCHIVE_COUNT} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
 
-	/** By default, submitting jobs from the web-frontend is allowed. */
+	/**
+	 * By default, submitting jobs from the web-frontend is allowed.
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_SUBMIT_ENABLE} instead
+	 */
+	@Deprecated
 	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMIT_ENABLED = true;
 
 	/** @deprecated Config key has been deprecated. Therefore, no default value required. */
 	@Deprecated
 	public static final boolean DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE = false;
 
-	/** Default number of checkpoints to remember for recent history. */
+	/**
+	 * Default number of checkpoints to remember for recent history.
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_CHECKPOINTS_HISTORY_SIZE} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE = 10;
 
-	/** Time after which cached stats are cleaned up. */
+	/**
+	 * Time after which cached stats are cleaned up.
+	 *
+	 * @@deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_CLEANUP_INTERVAL} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL = 10 * 60 * 1000;
 
-	/** Time after which available stats are deprecated and need to be refreshed (by resampling). */
+	/**
+	 * Time after which available stats are deprecated and need to be refreshed (by resampling).
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_REFRESH_INTERVAL} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL = 60 * 1000;
 
-	/** Number of samples to take to determine back pressure. */
+	/**
+	 * Number of samples to take to determine back pressure.
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_NUM_SAMPLES} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES = 100;
 
-	/** Delay between samples to determine back pressure. */
+	/**
+	 * Delay between samples to determine back pressure.
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_BACKPRESSURE_DELAY} instead
+	 */
+	@Deprecated
 	public static final int DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_DELAY = 50;
 
 	// ------------------------------ Akka Values ------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
index b924e8e..76b6bed 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
@@ -116,6 +116,13 @@ public class JobManagerOptions {
 			.defaultValue("*");
 
 	/**
+	 * The config parameter defining the refresh interval for the web-frontend.
+	 */
+	public static final ConfigOption<Long> WEB_REFRESH_INTERVAL =
+		key("jobmanager.web.refresh-interval")
+			.defaultValue(3000L);
+	
+	/**
 	 * Config parameter to override SSL support for the JobManager Web UI
 	 */
 	public static final ConfigOption<Boolean> WEB_SSL_ENABLED =

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index 709ef09..3d8a384 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -1235,10 +1235,6 @@ public abstract class ExecutionEnvironment {
 	public static ExecutionEnvironment createLocalEnvironmentWithWebUI(Configuration conf) {
 		checkNotNull(conf, "conf");
 
-		if (!conf.containsKey(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY)) {
-			int port = ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT;
-			conf.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, port);
-		}
 		conf.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true);
 
 		LocalEnvironment localEnv = new LocalEnvironment(conf);

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
index dba2145..77537a2 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
@@ -18,39 +18,11 @@
 
 package org.apache.flink.runtime.webmonitor;
 
-
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
 
 public class WebMonitorConfig {
 
-	// ------------------------------------------------------------------------
-	//  Config Keys
-	// ------------------------------------------------------------------------
-
-	/** The port for the runtime monitor web-frontend server. */
-	public static final String JOB_MANAGER_WEB_PORT_KEY = ConfigConstants.JOB_MANAGER_WEB_PORT_KEY;
-
-	/** The initial refresh interval for the web dashboard */
-	public static final String JOB_MANAGER_WEB_REFRESH_INTERVAL_KEY = "jobmanager.web.refresh-interval";
-
-
-	// ------------------------------------------------------------------------
-	//  Default values
-	// ------------------------------------------------------------------------
-
-	/** Default port for the web dashboard (= 8081) */
-	public static final int DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT = ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT;
-
-	/** Default refresh interval for the web dashboard (= 3000 msecs) */
-	public static final long DEFAULT_JOB_MANAGER_WEB_REFRESH_INTERVAL = 3000;
-
-
-	// ------------------------------------------------------------------------
-	//  Config
-	// ------------------------------------------------------------------------
-
 	/** The configuration queried by this config object */
 	private final Configuration config;
 
@@ -67,17 +39,15 @@ public class WebMonitorConfig {
 	}
 
 	public int getWebFrontendPort() {
-		return config.getInteger(JOB_MANAGER_WEB_PORT_KEY, DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT);
+		return config.getInteger(JobManagerOptions.WEB_PORT);
 	}
 
 	public long getRefreshInterval() {
-		return config.getLong(JOB_MANAGER_WEB_REFRESH_INTERVAL_KEY, DEFAULT_JOB_MANAGER_WEB_REFRESH_INTERVAL);
+		return config.getLong(JobManagerOptions.WEB_REFRESH_INTERVAL);
 	}
 	
 	public boolean isProgramSubmitEnabled() {
-		return config.getBoolean(
-			ConfigConstants.JOB_MANAGER_WEB_SUBMIT_ENABLED_KEY,
-			ConfigConstants.DEFAULT_JOB_MANAGER_WEB_SUBMIT_ENABLED);
+		return config.getBoolean(JobManagerOptions.WEB_SUBMIT_ENABLE);
 	}
 
 	public String getAllowOrigin() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/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 f83fa27..03b53ad 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
@@ -25,6 +25,7 @@ import io.netty.handler.codec.http.router.Router;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -191,21 +192,13 @@ public class WebRuntimeMonitor implements WebMonitor {
 		stackTraceSamples = new StackTraceSampleCoordinator(actorSystem.dispatcher(), 60000);
 
 		// Back pressure stats tracker config
-		int cleanUpInterval = config.getInteger(
-				ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL,
-				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL);
+		int cleanUpInterval = config.getInteger(JobManagerOptions.WEB_BACKPRESSURE_CLEANUP_INTERVAL);
 
-		int refreshInterval = config.getInteger(
-				ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL,
-				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL);
+		int refreshInterval = config.getInteger(JobManagerOptions.WEB_BACKPRESSURE_REFRESH_INTERVAL);
 
-		int numSamples = config.getInteger(
-				ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES,
-				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES);
+		int numSamples = config.getInteger(JobManagerOptions.WEB_BACKPRESSURE_NUM_SAMPLES);
 
-		int delay = config.getInteger(
-				ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_DELAY,
-				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_DELAY);
+		int delay = config.getInteger(JobManagerOptions.WEB_BACKPRESSURE_DELAY);
 
 		Time delayBetweenSamples = Time.milliseconds(delay);
 
@@ -219,10 +212,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 		ExecutionContextExecutor context = ExecutionContext$.MODULE$.fromExecutor(executorService);
 
 		// Config to enable https access to the web-ui
-		boolean enableSSL = config.getBoolean(
-				ConfigConstants.JOB_MANAGER_WEB_SSL_ENABLED,
-				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_SSL_ENABLED) &&
-			SSLUtils.getSSLEnabled(config);
+		boolean enableSSL = config.getBoolean(JobManagerOptions.WEB_SSL_ENABLED) &&	SSLUtils.getSSLEnabled(config);
 
 		if (enableSSL) {
 			LOG.info("Enabling ssl for the web frontend");
@@ -310,9 +300,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 		// DELETE is the preferred way of stopping a job (Rest-conform)
 		DELETE(router, new JobStoppingHandler());
 
-		int maxCachedEntries = config.getInteger(
-				ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
-			ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
+		int maxCachedEntries = config.getInteger(JobManagerOptions.WEB_CHECKPOINTS_HISTORY_SIZE);
 		CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries);
 
 		// Register the checkpoint stats handlers
@@ -525,14 +513,14 @@ public class WebRuntimeMonitor implements WebMonitor {
 	}
 
 	private String getBaseDirStr(Configuration configuration) {
-		return configuration.getString(ConfigConstants.JOB_MANAGER_WEB_TMPDIR_KEY, System.getProperty("java.io.tmpdir"));
+		return configuration.getString(JobManagerOptions.WEB_TMP_DIR);
 	}
 
 	private File getUploadDir(Configuration configuration) {
-		File baseDir = new File(configuration.getString(ConfigConstants.JOB_MANAGER_WEB_UPLOAD_DIR_KEY,
+		File baseDir = new File(configuration.getString(JobManagerOptions.WEB_UPLOAD_DIR,
 			getBaseDirStr(configuration)));
 
-		boolean uploadDirSpecified = configuration.containsKey(ConfigConstants.JOB_MANAGER_WEB_UPLOAD_DIR_KEY);
+		boolean uploadDirSpecified = configuration.contains(JobManagerOptions.WEB_UPLOAD_DIR);
 		return uploadDirSpecified ? baseDir : new File(baseDir, "flink-web-" + UUID.randomUUID());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 5ccfe90..a51a234 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -25,6 +25,7 @@ import org.apache.curator.test.TestingServer;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
@@ -137,8 +138,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 			Path logFile = Files.createFile(new File(logDir, "jobmanager.log").toPath());
 			Files.createFile(new File(logDir, "jobmanager.out").toPath());
 
-			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
-			config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
+			config.setInteger(JobManagerOptions.WEB_PORT, 0);
+			config.setString(JobManagerOptions.WEB_LOG_PATH, logFile.toString());
 
 			highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
 				config,
@@ -286,8 +287,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 			Files.createFile(new File(logDir, "jobmanager.out").toPath());
 
 			final Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
-			config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
+			config.setInteger(JobManagerOptions.WEB_PORT, 0);
+			config.setString(JobManagerOptions.WEB_LOG_PATH, logFile.toString());
 			config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
 			config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeper.getConnectString());
 
@@ -463,8 +464,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 
 		// Web frontend on random port
 		Configuration config = new Configuration();
-		config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
-		config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
+		config.setInteger(JobManagerOptions.WEB_PORT, 0);
+		config.setString(JobManagerOptions.WEB_LOG_PATH, logFile.toString());
 
 		WebRuntimeMonitor webMonitor = new WebRuntimeMonitor(
 			config,

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
index de0d9d0..0e76cfb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
@@ -205,9 +206,7 @@ public class ExecutionGraphBuilder {
 			}
 
 			// Maximum number of remembered checkpoints
-			int historySize = jobManagerConfig.getInteger(
-					ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
-					ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
+			int historySize = jobManagerConfig.getInteger(JobManagerOptions.WEB_CHECKPOINTS_HISTORY_SIZE);
 
 			CheckpointStatsTracker checkpointStatsTracker = new CheckpointStatsTracker(
 					historySize,

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
index 2baadb5..dd9527e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
@@ -25,8 +25,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 
 import java.net.URI;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
@@ -80,14 +80,14 @@ public final class WebMonitorUtils {
 			
 			if (logFilePath == null) {
 				LOG.warn("Log file environment variable '{}' is not set.", logEnv);
-				logFilePath = config.getString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, null);
+				logFilePath = config.getString(JobManagerOptions.WEB_LOG_PATH);
 			}
 			
 			// not configured, cannot serve log files
 			if (logFilePath == null || logFilePath.length() < 4) {
 				LOG.warn("JobManager log files are unavailable in the web dashboard. " +
 					"Log file location not found in environment variable '{}' or configuration key '{}'.",
-					logEnv, ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY);
+					logEnv, JobManagerOptions.WEB_LOG_PATH.key());
 				return new LogFileLocation(null, null);
 			}
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 5092643..57a6415 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -2532,8 +2532,7 @@ object JobManager {
 
     val restartStrategy = RestartStrategyFactory.createRestartStrategyFactory(configuration)
 
-    val archiveCount = configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_ARCHIVE_COUNT,
-      ConfigConstants.DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT)
+    val archiveCount = configuration.getInteger(JobManagerOptions.WEB_ARCHIVE_COUNT)
 
     val archiveDir = configuration.getString(JobManagerOptions.ARCHIVE_DIR)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index aad3a4b..97117d2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -1661,10 +1661,6 @@ public abstract class StreamExecutionEnvironment {
 	public static StreamExecutionEnvironment createLocalEnvironmentWithWebUI(Configuration conf) {
 		checkNotNull(conf, "conf");
 
-		if (!conf.containsKey(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY)) {
-			int port = ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT;
-			conf.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, port);
-		}
 		conf.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true);
 
 		LocalStreamEnvironment localEnv = new LocalStreamEnvironment(conf);

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index f96ab3d..437dd5f 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -30,6 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
@@ -148,8 +149,8 @@ public class TestBaseUtils extends TestLogger {
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, DEFAULT_AKKA_ASK_TIMEOUT + "s");
 		config.setString(ConfigConstants.AKKA_STARTUP_TIMEOUT, DEFAULT_AKKA_STARTUP_TIMEOUT);
 
-		config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 8081);
-		config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
+		config.setInteger(JobManagerOptions.WEB_PORT, 8081);
+		config.setString(JobManagerOptions.WEB_LOG_PATH, logFile.toString());
 
 		config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.toString());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
index 003eb0c..538ac98 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
@@ -24,12 +24,15 @@ import com.fasterxml.jackson.databind.node.ArrayNode;
 
 import org.apache.commons.io.FileUtils;
 
+import org.apache.commons.math3.optim.nonlinear.vector.JacobianMultivariateVectorOptimizer;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobManagerCliOptions;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testutils.StoppableInvokable;
 import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
@@ -84,7 +87,7 @@ public class WebFrontendITCase extends TestLogger {
 		Files.createFile(logFile.toPath());
 		Files.createFile(outFile.toPath());
 		
-		config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.getAbsolutePath());
+		config.setString(JobManagerOptions.WEB_LOG_PATH, logFile.getAbsolutePath());
 		config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.getAbsolutePath());
 
 		cluster = new LocalFlinkMiniCluster(config, false);

http://git-wip-us.apache.org/repos/asf/flink/blob/9708550a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
index b62f957..64417f6 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
@@ -372,8 +372,7 @@ public class YarnApplicationMasterRunner {
 				LOG);
 
 			String protocol = "http://";
-			if (config.getBoolean(ConfigConstants.JOB_MANAGER_WEB_SSL_ENABLED,
-				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
+			if (config.getBoolean(JobManagerOptions.WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
 				protocol = "https://";
 			}
 			final String webMonitorURL = webMonitor == null ? null :


[06/14] flink git commit: [FLINK-6537] [checkpoint] First set of fixes for (de)registration of shared state in incremental checkpoints

Posted by sr...@apache.org.
[FLINK-6537] [checkpoint] First set of fixes for (de)registration of shared state in incremental checkpoints


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

Branch: refs/heads/release-1.3
Commit: b30b8eef044bd2af27485cd7c09b9b48dadd9dd8
Parents: 5e61a01
Author: Stefan Richter <s....@data-artisans.com>
Authored: Wed May 10 17:59:39 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../RocksDBIncrementalKeyedStateHandle.java     | 123 ++++++------
 .../state/RocksDBKeyedStateBackend.java         |  64 +++++-
 .../runtime/state/SharedStateRegistry.java      | 196 ++++++++++++++-----
 .../runtime/state/SharedStateRegistryKey.java   |  68 +++++++
 .../runtime/state/SharedStateRegistryTest.java  |  85 +++++---
 .../runtime/state/StateBackendTestBase.java     |  18 +-
 6 files changed, 397 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b30b8eef/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
index 5ac9e46..961182d 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
@@ -18,12 +18,11 @@
 
 package org.apache.flink.contrib.streaming.state;
 
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.state.CompositeStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
-import org.apache.flink.runtime.state.SharedStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.SharedStateRegistryKey;
 import org.apache.flink.runtime.state.StateUtil;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.util.Preconditions;
@@ -54,17 +53,15 @@ public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, Com
 
 	private static final long serialVersionUID = -8328808513197388231L;
 
-	private final JobID jobId;
-
 	private final String operatorIdentifier;
 
 	private final KeyGroupRange keyGroupRange;
 
 	private final long checkpointId;
 
-	private final Map<String, StreamStateHandle> newSstFiles;
+	private final Map<String, StreamStateHandle> unregisteredSstFiles;
 
-	private final Map<String, StreamStateHandle> oldSstFiles;
+	private final Map<String, StreamStateHandle> registeredSstFiles;
 
 	private final Map<String, StreamStateHandle> miscFiles;
 
@@ -81,21 +78,19 @@ public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, Com
 	private boolean registered;
 
 	RocksDBIncrementalKeyedStateHandle(
-			JobID jobId,
 			String operatorIdentifier,
 			KeyGroupRange keyGroupRange,
 			long checkpointId,
-			Map<String, StreamStateHandle> newSstFiles,
-			Map<String, StreamStateHandle> oldSstFiles,
+			Map<String, StreamStateHandle> unregisteredSstFiles,
+			Map<String, StreamStateHandle> registeredSstFiles,
 			Map<String, StreamStateHandle> miscFiles,
 			StreamStateHandle metaStateHandle) {
 
-		this.jobId = Preconditions.checkNotNull(jobId);
 		this.operatorIdentifier = Preconditions.checkNotNull(operatorIdentifier);
 		this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
 		this.checkpointId = checkpointId;
-		this.newSstFiles = Preconditions.checkNotNull(newSstFiles);
-		this.oldSstFiles = Preconditions.checkNotNull(oldSstFiles);
+		this.unregisteredSstFiles = Preconditions.checkNotNull(unregisteredSstFiles);
+		this.registeredSstFiles = Preconditions.checkNotNull(registeredSstFiles);
 		this.miscFiles = Preconditions.checkNotNull(miscFiles);
 		this.metaStateHandle = Preconditions.checkNotNull(metaStateHandle);
 		this.registered = false;
@@ -110,12 +105,12 @@ public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, Com
 		return checkpointId;
 	}
 
-	Map<String, StreamStateHandle> getNewSstFiles() {
-		return newSstFiles;
+	Map<String, StreamStateHandle> getUnregisteredSstFiles() {
+		return unregisteredSstFiles;
 	}
 
-	Map<String, StreamStateHandle> getOldSstFiles() {
-		return oldSstFiles;
+	Map<String, StreamStateHandle> getRegisteredSstFiles() {
+		return registeredSstFiles;
 	}
 
 	Map<String, StreamStateHandle> getMiscFiles() {
@@ -138,6 +133,8 @@ public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, Com
 	@Override
 	public void discardState() throws Exception {
 
+		Preconditions.checkState(!registered, "Attempt to dispose a registered composite state with registered shared state. Must unregister first.");
+
 		try {
 			metaStateHandle.discardState();
 		} catch (Exception e) {
@@ -150,24 +147,23 @@ public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, Com
 			LOG.warn("Could not properly discard misc file states.", e);
 		}
 
-		if (!registered) {
-			try {
-				StateUtil.bestEffortDiscardAllStateObjects(newSstFiles.values());
-			} catch (Exception e) {
-				LOG.warn("Could not properly discard new sst file states.", e);
-			}
+		try {
+			StateUtil.bestEffortDiscardAllStateObjects(unregisteredSstFiles.values());
+		} catch (Exception e) {
+			LOG.warn("Could not properly discard new sst file states.", e);
 		}
+
 	}
 
 	@Override
 	public long getStateSize() {
 		long size = StateUtil.getStateSize(metaStateHandle);
 
-		for (StreamStateHandle newSstFileHandle : newSstFiles.values()) {
+		for (StreamStateHandle newSstFileHandle : unregisteredSstFiles.values()) {
 			size += newSstFileHandle.getStateSize();
 		}
 
-		for (StreamStateHandle oldSstFileHandle : oldSstFiles.values()) {
+		for (StreamStateHandle oldSstFileHandle : registeredSstFiles.values()) {
 			size += oldSstFileHandle.getStateSize();
 		}
 
@@ -180,69 +176,66 @@ public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, Com
 
 	@Override
 	public void registerSharedStates(SharedStateRegistry stateRegistry) {
+
 		Preconditions.checkState(!registered, "The state handle has already registered its shared states.");
 
-		for (Map.Entry<String, StreamStateHandle> newSstFileEntry : newSstFiles.entrySet()) {
-			SstFileStateHandle stateHandle = new SstFileStateHandle(newSstFileEntry.getKey(), newSstFileEntry.getValue());
+		for (Map.Entry<String, StreamStateHandle> newSstFileEntry : unregisteredSstFiles.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(newSstFileEntry.getKey());
 
-			int referenceCount = stateRegistry.register(stateHandle);
-			Preconditions.checkState(referenceCount == 1);
+			SharedStateRegistry.Result result =
+				stateRegistry.registerNewReference(registryKey, newSstFileEntry.getValue());
+
+			// We update our reference with the result from the registry, to prevent the following
+			// problem:
+			// A previous checkpoint n has already registered the state. This can happen if a
+			// following checkpoint (n + x) wants to reference the same state before the backend got
+			// notified that checkpoint n completed. In this case, the shared registry did
+			// deduplication and returns the previous reference.
+			newSstFileEntry.setValue(result.getReference());
 		}
 
-		for (Map.Entry<String, StreamStateHandle> oldSstFileEntry : oldSstFiles.entrySet()) {
-			SstFileStateHandle stateHandle = new SstFileStateHandle(oldSstFileEntry.getKey(), oldSstFileEntry.getValue());
+		for (Map.Entry<String, StreamStateHandle> oldSstFileName : registeredSstFiles.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(oldSstFileName.getKey());
+
+			SharedStateRegistry.Result result = stateRegistry.obtainReference(registryKey);
 
-			int referenceCount = stateRegistry.register(stateHandle);
-			Preconditions.checkState(referenceCount > 1);
+			// Again we update our state handle with the result from the registry, thus replacing
+			// placeholder state handles with the originals.
+			oldSstFileName.setValue(result.getReference());
 		}
 
+		// Migrate state from unregistered to registered, so that it will not count as private state
+		// for #discardState() from now.
+		registeredSstFiles.putAll(unregisteredSstFiles);
+		unregisteredSstFiles.clear();
+
 		registered = true;
 	}
 
 	@Override
 	public void unregisterSharedStates(SharedStateRegistry stateRegistry) {
+
 		Preconditions.checkState(registered, "The state handle has not registered its shared states yet.");
 
-		for (Map.Entry<String, StreamStateHandle> newSstFileEntry : newSstFiles.entrySet()) {
-			stateRegistry.unregister(new SstFileStateHandle(newSstFileEntry.getKey(), newSstFileEntry.getValue()));
+		for (Map.Entry<String, StreamStateHandle> newSstFileEntry : unregisteredSstFiles.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(newSstFileEntry.getKey());
+			stateRegistry.releaseReference(registryKey);
 		}
 
-		for (Map.Entry<String, StreamStateHandle> oldSstFileEntry : oldSstFiles.entrySet()) {
-			stateRegistry.unregister(new SstFileStateHandle(oldSstFileEntry.getKey(), oldSstFileEntry.getValue()));
+		for (Map.Entry<String, StreamStateHandle> oldSstFileEntry : registeredSstFiles.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(oldSstFileEntry.getKey());
+			stateRegistry.releaseReference(registryKey);
 		}
 
 		registered = false;
 	}
 
-	private class SstFileStateHandle implements SharedStateHandle {
-
-		private static final long serialVersionUID = 9092049285789170669L;
-
-		private final String fileName;
-
-		private final StreamStateHandle delegateStateHandle;
-
-		private SstFileStateHandle(
-				String fileName,
-				StreamStateHandle delegateStateHandle) {
-			this.fileName = fileName;
-			this.delegateStateHandle = delegateStateHandle;
-		}
-
-		@Override
-		public String getRegistrationKey() {
-			return jobId + "-" + operatorIdentifier + "-" + keyGroupRange + "-" + fileName;
-		}
-
-		@Override
-		public void discardState() throws Exception {
-			delegateStateHandle.discardState();
-		}
-
-		@Override
-		public long getStateSize() {
-			return delegateStateHandle.getStateSize();
-		}
+	private SharedStateRegistryKey createSharedStateRegistryKeyFromFileName(String fileName) {
+		return new SharedStateRegistryKey(operatorIdentifier + "-" + keyGroupRange + "-" + fileName);
 	}
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b30b8eef/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 6af53c3..1080e59 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -54,7 +54,6 @@ import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
-import org.apache.flink.runtime.state.StateMigrationUtil;
 import org.apache.flink.runtime.state.DoneFuture;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
@@ -62,9 +61,10 @@ import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
 import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
+import org.apache.flink.runtime.state.StateMigrationUtil;
 import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.StateUtil;
-import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.internal.InternalAggregatingState;
 import org.apache.flink.runtime.state.internal.InternalFoldingState;
@@ -72,6 +72,7 @@ import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.runtime.state.internal.InternalReducingState;
 import org.apache.flink.runtime.state.internal.InternalValueState;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
 import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.IOUtils;
@@ -709,16 +710,22 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	private static final class RocksDBIncrementalSnapshotOperation {
 
+		/** The backend which we snapshot */
 		private final RocksDBKeyedStateBackend<?> stateBackend;
 
+		/** Stream factory that creates the outpus streams to DFS */
 		private final CheckpointStreamFactory checkpointStreamFactory;
 
+		/** Id for the current checkpoint */
 		private final long checkpointId;
 
+		/** Timestamp for the current checkpoint */
 		private final long checkpointTimestamp;
 
+		/** All sst files that were part of the last previously completed checkpoint */
 		private Map<String, StreamStateHandle> baseSstFiles;
 
+		/** The state meta data */
 		private final List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> stateMetaInfoSnapshots = new ArrayList<>();
 
 		private FileSystem backupFileSystem;
@@ -864,10 +871,11 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 						if (fileHandle == null) {
 							fileHandle = materializeStateData(filePath);
-
 							newSstFiles.put(fileName, fileHandle);
 						} else {
-							oldSstFiles.put(fileName, fileHandle);
+							// we introduce a placeholder state handle, that is replaced with the
+							// original from the shared state registry (created from a previous checkpoint)
+							oldSstFiles.put(fileName, new PlaceholderStreamStateHandle(fileHandle.getStateSize()));
 						}
 					} else {
 						StreamStateHandle fileHandle = materializeStateData(filePath);
@@ -882,9 +890,14 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 			stateBackend.materializedSstFiles.put(checkpointId, sstFiles);
 
-			return new RocksDBIncrementalKeyedStateHandle(stateBackend.jobId,
-				stateBackend.operatorIdentifier, stateBackend.keyGroupRange,
-				checkpointId, newSstFiles, oldSstFiles, miscFiles, metaStateHandle);
+			return new RocksDBIncrementalKeyedStateHandle(
+				stateBackend.operatorIdentifier,
+				stateBackend.keyGroupRange,
+				checkpointId,
+				newSstFiles,
+				oldSstFiles,
+				miscFiles,
+				metaStateHandle);
 		}
 
 		void stop() {
@@ -922,6 +935,39 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				}
 			}
 		}
+
+		/**
+		 * A placeholder state handle for shared state that will replaced by an original that was
+		 * created in a previous checkpoint. So we don't have to send the handle twice, e.g. in
+		 * case of {@link ByteStreamStateHandle}.
+		 */
+		private static final class PlaceholderStreamStateHandle implements StreamStateHandle {
+
+			private static final long serialVersionUID = 1L;
+
+			/** We remember the size of the original file for which this is a placeholder */
+			private final long originalSize;
+
+			public PlaceholderStreamStateHandle(long originalSize) {
+				this.originalSize = originalSize;
+			}
+
+			@Override
+			public FSDataInputStream openInputStream() {
+				throw new UnsupportedOperationException(
+					"This is only a placeholder to be replaced by a real StreamStateHandle in the checkpoint coordinator.");
+			}
+
+			@Override
+			public void discardState() throws Exception {
+				// nothing to do.
+			}
+
+			@Override
+			public long getStateSize() {
+				return originalSize;
+			}
+		}
 	}
 
 	@Override
@@ -1260,7 +1306,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				UUID.randomUUID().toString());
 
 			try {
-				Map<String, StreamStateHandle> newSstFiles = restoreStateHandle.getNewSstFiles();
+				Map<String, StreamStateHandle> newSstFiles = restoreStateHandle.getUnregisteredSstFiles();
 				for (Map.Entry<String, StreamStateHandle> newSstFileEntry : newSstFiles.entrySet()) {
 					String fileName = newSstFileEntry.getKey();
 					StreamStateHandle remoteFileHandle = newSstFileEntry.getValue();
@@ -1268,7 +1314,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 					readStateData(new Path(restoreInstancePath, fileName), remoteFileHandle);
 				}
 
-				Map<String, StreamStateHandle> oldSstFiles = restoreStateHandle.getOldSstFiles();
+				Map<String, StreamStateHandle> oldSstFiles = restoreStateHandle.getRegisteredSstFiles();
 				for (Map.Entry<String, StreamStateHandle> oldSstFileEntry : oldSstFiles.entrySet()) {
 					String fileName = oldSstFileEntry.getKey();
 					StreamStateHandle remoteFileHandle = oldSstFileEntry.getValue();

http://git-wip-us.apache.org/repos/asf/flink/blob/b30b8eef/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
index 2cb43ac..dbf4642 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
@@ -18,91 +18,137 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.Executor;
 
 /**
  * A {@code SharedStateRegistry} will be deployed in the 
- * {@link org.apache.flink.runtime.checkpoint.CheckpointCoordinator} to 
+ * {@link org.apache.flink.runtime.checkpoint.CompletedCheckpointStore} to
  * maintain the reference count of {@link SharedStateHandle}s which are shared
- * among different checkpoints.
- *
+ * among different incremental checkpoints.
  */
 public class SharedStateRegistry {
 
 	private static final Logger LOG = LoggerFactory.getLogger(SharedStateRegistry.class);
 
 	/** All registered state objects by an artificial key */
-	private final Map<String, SharedStateRegistry.SharedStateEntry> registeredStates;
+	private final Map<SharedStateRegistryKey, SharedStateRegistry.SharedStateEntry> registeredStates;
+
+	/** Executor for async state deletion */
+	private final Executor asyncDisposalExecutor;
 
 	public SharedStateRegistry() {
 		this.registeredStates = new HashMap<>();
+		this.asyncDisposalExecutor = Executors.directExecutor(); //TODO: FLINK-6534
 	}
 
 	/**
-	 * Register a reference to the given shared state in the registry. This increases the reference
-	 * count for the this shared state by one. Returns the reference count after the update.
+	 * Register a reference to the given (supposedly new) shared state in the registry.
+	 * This does the following: We check if the state handle is actually new by the
+	 * registrationKey. If it is new, we register it with a reference count of 1. If there is
+	 * already a state handle registered under the given key, we dispose the given "new" state
+	 * handle, uptick the reference count of the previously existing state handle and return it as
+	 * a replacement with the result.
+	 *
+	 * <p>IMPORTANT: caller should check the state handle returned by the result, because the
+	 * registry is performing deduplication and could potentially return a handle that is supposed
+	 * to replace the one from the registration request.
 	 *
 	 * @param state the shared state for which we register a reference.
-	 * @return the updated reference count for the given shared state.
+	 * @return the result of this registration request, consisting of the state handle that is
+	 * registered under the key by the end of the oepration and its current reference count.
 	 */
-	public int register(SharedStateHandle state) {
-		if (state == null) {
-			return 0;
-		}
+	public Result registerNewReference(SharedStateRegistryKey registrationKey, StreamStateHandle state) {
+
+		Preconditions.checkNotNull(state);
+
+		StreamStateHandle scheduledStateDeletion = null;
+		SharedStateRegistry.SharedStateEntry entry;
 
 		synchronized (registeredStates) {
-			SharedStateRegistry.SharedStateEntry entry =
-				registeredStates.get(state.getRegistrationKey());
+			entry = registeredStates.get(registrationKey);
 
 			if (entry == null) {
-				SharedStateRegistry.SharedStateEntry stateEntry =
-					new SharedStateRegistry.SharedStateEntry(state);
-				registeredStates.put(state.getRegistrationKey(), stateEntry);
-				return 1;
+				entry = new SharedStateRegistry.SharedStateEntry(state);
+				registeredStates.put(registrationKey, entry);
 			} else {
+				// delete if this is a real duplicate
+				if (!Objects.equals(state, entry.state)) {
+					scheduledStateDeletion = state;
+				}
 				entry.increaseReferenceCount();
-				return entry.getReferenceCount();
 			}
 		}
+
+		scheduleAsyncDelete(scheduledStateDeletion);
+		return new Result(entry);
 	}
 
 	/**
-	 * Unregister one reference to the given shared state in the registry. This decreases the
-	 * reference count by one. Once the count reaches zero, the shared state is deleted.
+	 * Obtains one reference to the given shared state in the registry. This increases the
+	 * reference count by one.
 	 *
-	 * @param state the shared state for which we unregister a reference.
-	 * @return the reference count for the shared state after the update.
+	 * @param registrationKey the shared state for which we obtain a reference.
+	 * @return the shared state for which we release a reference.
+	 * @return the result of the request, consisting of the reference count after this operation
+	 * and the state handle.
 	 */
-	public int unregister(SharedStateHandle state) {
-		if (state == null) {
-			return 0;
+	public Result obtainReference(SharedStateRegistryKey registrationKey) {
+
+		Preconditions.checkNotNull(registrationKey);
+
+		synchronized (registeredStates) {
+			SharedStateRegistry.SharedStateEntry entry =
+				Preconditions.checkNotNull(registeredStates.get(registrationKey),
+					"Could not find a state for the given registration key!");
+			entry.increaseReferenceCount();
+			return new Result(entry);
 		}
+	}
+
+	/**
+	 * Releases one reference to the given shared state in the registry. This decreases the
+	 * reference count by one. Once the count reaches zero, the shared state is deleted.
+	 *
+	 * @param registrationKey the shared state for which we release a reference.
+	 * @return the result of the request, consisting of the reference count after this operation
+	 * and the state handle, or null if the state handle was deleted through this request.
+	 */
+	public Result releaseReference(SharedStateRegistryKey registrationKey) {
+
+		Preconditions.checkNotNull(registrationKey);
+
+		final Result result;
+		final StreamStateHandle scheduledStateDeletion;
 
 		synchronized (registeredStates) {
-			SharedStateRegistry.SharedStateEntry entry = registeredStates.get(state.getRegistrationKey());
+			SharedStateRegistry.SharedStateEntry entry = registeredStates.get(registrationKey);
 
-			Preconditions.checkState(entry != null, "Cannot unregister a state that is not registered.");
+			Preconditions.checkState(entry != null,
+				"Cannot unregister a state that is not registered.");
 
 			entry.decreaseReferenceCount();
 
-			final int newReferenceCount = entry.getReferenceCount();
-
 			// Remove the state from the registry when it's not referenced any more.
-			if (newReferenceCount <= 0) {
-				registeredStates.remove(state.getRegistrationKey());
-				try {
-					entry.getState().discardState();
-				} catch (Exception e) {
-					LOG.warn("Cannot properly discard the state {}.", entry.getState(), e);
-				}
+			if (entry.getReferenceCount() <= 0) {
+				registeredStates.remove(registrationKey);
+				scheduledStateDeletion = entry.getState();
+				result = new Result(null, 0);
+			} else {
+				scheduledStateDeletion = null;
+				result = new Result(entry);
 			}
-			return newReferenceCount;
 		}
+
+		scheduleAsyncDelete(scheduledStateDeletion);
+		return result;
 	}
 
 	/**
@@ -122,8 +168,6 @@ public class SharedStateRegistry {
 		}
 	}
 
-
-
 	/**
 	 * Unregister all the shared states referenced by the given.
 	 *
@@ -141,20 +185,30 @@ public class SharedStateRegistry {
 		}
 	}
 
+	private void scheduleAsyncDelete(StreamStateHandle streamStateHandle) {
+		if (streamStateHandle != null) {
+			asyncDisposalExecutor.execute(
+				new SharedStateRegistry.AsyncDisposalRunnable(streamStateHandle));
+		}
+	}
+
+	/**
+	 * An entry in the registry, tracking the handle and the corresponding reference count.
+	 */
 	private static class SharedStateEntry {
 
-		/** The shared object */
-		private final SharedStateHandle state;
+		/** The shared state handle */
+		private final StreamStateHandle state;
 
-		/** The reference count of the object */
+		/** The current reference count of the state handle */
 		private int referenceCount;
 
-		SharedStateEntry(SharedStateHandle value) {
+		SharedStateEntry(StreamStateHandle value) {
 			this.state = value;
 			this.referenceCount = 1;
 		}
 
-		SharedStateHandle getState() {
+		StreamStateHandle getState() {
 			return state;
 		}
 
@@ -171,14 +225,56 @@ public class SharedStateRegistry {
 		}
 	}
 
-	public int getReferenceCount(SharedStateHandle state) {
-		if (state == null) {
-			return 0;
+	/**
+	 * The result of an attempt to (un)/reference state
+	 */
+	public static class Result {
+
+		/** The (un)registered state handle from the request */
+		private final StreamStateHandle reference;
+
+		/** The reference count to the state handle after the request to (un)register */
+		private final int referenceCount;
+
+		private Result(SharedStateEntry sharedStateEntry) {
+			this.reference = sharedStateEntry.getState();
+			this.referenceCount = sharedStateEntry.getReferenceCount();
 		}
 
-		SharedStateRegistry.SharedStateEntry entry =
-			registeredStates.get(state.getRegistrationKey());
+		public Result(StreamStateHandle reference, int referenceCount) {
+			Preconditions.checkArgument(referenceCount >= 0);
 
-		return entry == null ? 0 : entry.getReferenceCount();
+			this.reference = reference;
+			this.referenceCount = referenceCount;
+		}
+
+		public StreamStateHandle getReference() {
+			return reference;
+		}
+
+		public int getReferenceCount() {
+			return referenceCount;
+		}
+	}
+
+	/**
+	 * Encapsulates the operation the delete state handles asynchronously.
+	 */
+	private static final class AsyncDisposalRunnable implements Runnable {
+
+		private final StateObject toDispose;
+
+		public AsyncDisposalRunnable(StateObject toDispose) {
+			this.toDispose = Preconditions.checkNotNull(toDispose);
+		}
+
+		@Override
+		public void run() {
+			try {
+				toDispose.discardState();
+			} catch (Exception e) {
+				LOG.warn("A problem occurred during asynchronous disposal of a shared state object: {}", toDispose, e);
+			}
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b30b8eef/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
new file mode 100644
index 0000000..9e59359
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * This class represents a key that uniquely identifies (on a logical level) state handles for
+ * registration in the {@link SharedStateRegistry}. Two files which should logically
+ * be the same should have the same {@link SharedStateRegistryKey}. The meaning of logical
+ * equivalence is up to the application.
+ */
+public class SharedStateRegistryKey implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	/** Uses a String as internal representation */
+	private final String keyString;
+
+	public SharedStateRegistryKey(String keyString) {
+		this.keyString = Preconditions.checkNotNull(keyString);
+	}
+
+	public String getKeyString() {
+		return keyString;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SharedStateRegistryKey that = (SharedStateRegistryKey) o;
+		return keyString.equals(that.keyString);
+	}
+
+	@Override
+	public int hashCode() {
+		return keyString.hashCode();
+	}
+
+	@Override
+	public String toString() {
+		return keyString;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b30b8eef/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedStateRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedStateRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedStateRegistryTest.java
index 821bb69..03e2a13 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedStateRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedStateRegistryTest.java
@@ -19,9 +19,14 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.core.fs.FSDataInputStream;
 import org.junit.Test;
 
+import java.io.IOException;
+
+import static junit.framework.TestCase.assertFalse;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class SharedStateRegistryTest {
 
@@ -30,24 +35,50 @@ public class SharedStateRegistryTest {
 	 */
 	@Test
 	public void testRegistryNormal() {
+
 		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 
 		// register one state
 		TestSharedState firstState = new TestSharedState("first");
-		assertEquals(1, sharedStateRegistry.register(firstState));
+		SharedStateRegistry.Result result = sharedStateRegistry.registerNewReference(firstState.getRegistrationKey(), firstState);
+		assertEquals(1, result.getReferenceCount());
+		assertTrue(firstState == result.getReference());
+		assertFalse(firstState.isDiscarded());
 
 		// register another state
 		TestSharedState secondState = new TestSharedState("second");
-		assertEquals(1, sharedStateRegistry.register(secondState));
-
-		// register the first state again
-		assertEquals(2, sharedStateRegistry.register(firstState));
+		result = sharedStateRegistry.registerNewReference(secondState.getRegistrationKey(), secondState);
+		assertEquals(1, result.getReferenceCount());
+		assertTrue(secondState == result.getReference());
+		assertFalse(firstState.isDiscarded());
+		assertFalse(secondState.isDiscarded());
+
+		// attempt to register state under an existing key
+		TestSharedState firstStatePrime = new TestSharedState(firstState.getRegistrationKey().getKeyString());
+		result = sharedStateRegistry.registerNewReference(firstState.getRegistrationKey(), firstStatePrime);
+		assertEquals(2, result.getReferenceCount());
+		assertFalse(firstStatePrime == result.getReference());
+		assertTrue(firstState == result.getReference());
+		assertTrue(firstStatePrime.isDiscarded());
+		assertFalse(firstState.isDiscarded());
+
+		// reference the first state again
+		result = sharedStateRegistry.obtainReference(firstState.getRegistrationKey());
+		assertEquals(3, result.getReferenceCount());
+		assertTrue(firstState == result.getReference());
+		assertFalse(firstState.isDiscarded());
 
 		// unregister the second state
-		assertEquals(0, sharedStateRegistry.unregister(secondState));
+		result = sharedStateRegistry.releaseReference(secondState.getRegistrationKey());
+		assertEquals(0, result.getReferenceCount());
+		assertTrue(result.getReference() == null);
+		assertTrue(secondState.isDiscarded());
 
 		// unregister the first state
-		assertEquals(1, sharedStateRegistry.unregister(firstState));
+		result = sharedStateRegistry.releaseReference(firstState.getRegistrationKey());
+		assertEquals(2, result.getReferenceCount());
+		assertTrue(firstState == result.getReference());
+		assertFalse(firstState.isDiscarded());
 	}
 
 	/**
@@ -56,51 +87,47 @@ public class SharedStateRegistryTest {
 	@Test(expected = IllegalStateException.class)
 	public void testUnregisterWithUnexistedKey() {
 		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
-
-		sharedStateRegistry.unregister(new TestSharedState("unexisted"));
+		sharedStateRegistry.releaseReference(new SharedStateRegistryKey("non-existent"));
 	}
 
-	private static class TestSharedState implements SharedStateHandle {
+	private static class TestSharedState implements StreamStateHandle {
 		private static final long serialVersionUID = 4468635881465159780L;
 
-		private String key;
+		private SharedStateRegistryKey key;
+
+		private boolean discarded;
 
 		TestSharedState(String key) {
-			this.key = key;
+			this.key = new SharedStateRegistryKey(key);
+			this.discarded = false;
 		}
 
-		@Override
-		public String getRegistrationKey() {
+		public SharedStateRegistryKey getRegistrationKey() {
 			return key;
 		}
 
 		@Override
 		public void discardState() throws Exception {
-			// nothing to do
+			this.discarded = true;
 		}
 
 		@Override
 		public long getStateSize() {
-			return key.length();
+			return key.toString().length();
 		}
 
 		@Override
-		public boolean equals(Object o) {
-			if (this == o) {
-				return true;
-			}
-			if (o == null || getClass() != o.getClass()) {
-				return false;
-			}
-
-			TestSharedState testState = (TestSharedState) o;
-
-			return key.equals(testState.key);
+		public int hashCode() {
+			return key.hashCode();
 		}
 
 		@Override
-		public int hashCode() {
-			return key.hashCode();
+		public FSDataInputStream openInputStream() throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		public boolean isDiscarded() {
+			return discarded;
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b30b8eef/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index 658ccde..ca66ffb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -482,6 +482,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	@SuppressWarnings("unchecked")
 	public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws Exception {
 		CheckpointStreamFactory streamFactory = createStreamFactory();
+		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 		Environment env = new DummyEnvironment("test", 1, 0);
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
 
@@ -509,6 +510,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
+		snapshot.registerSharedStates(sharedStateRegistry);
 		backend.dispose();
 
 		// ========== restore snapshot - should use default serializer (ONLY SERIALIZATION) ==========
@@ -518,8 +520,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
 
-		snapshot.discardState();
-
 		// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
 		// initializeSerializerUnlessSet would not pick up our new config
 		kvId = new ValueStateDescriptor<>("id", pojoType);
@@ -536,6 +536,11 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
+		snapshot2.registerSharedStates(sharedStateRegistry);
+
+		snapshot.unregisterSharedStates(sharedStateRegistry);
+		snapshot.discardState();
+
 		backend.dispose();
 
 		// ========= restore snapshot - should use default serializer (FAIL ON DESERIALIZATION) =========
@@ -570,6 +575,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	@Test
 	public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception {
 		CheckpointStreamFactory streamFactory = createStreamFactory();
+		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 		Environment env = new DummyEnvironment("test", 1, 0);
 
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
@@ -597,6 +603,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
+		snapshot.registerSharedStates(sharedStateRegistry);
 		backend.dispose();
 
 		// ========== restore snapshot - should use specific serializer (ONLY SERIALIZATION) ==========
@@ -605,8 +612,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
 
-		snapshot.discardState();
-
 		// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
 		// initializeSerializerUnlessSet would not pick up our new config
 		kvId = new ValueStateDescriptor<>("id", pojoType);
@@ -623,6 +628,11 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
+		snapshot2.registerSharedStates(sharedStateRegistry);
+
+		snapshot.unregisterSharedStates(sharedStateRegistry);
+		snapshot.discardState();
+
 		backend.dispose();
 
 		// ========= restore snapshot - should use specific serializer (FAIL ON DESERIALIZATION) =========


[09/14] flink git commit: [FLINK-6414] [build] Use scala.binary.version in place of change-scala-version.sh

Posted by sr...@apache.org.
[FLINK-6414] [build] Use scala.binary.version in place of change-scala-version.sh

Use scala.binary.version as defined in the parent POM and remove the
script to swap scala version identifiers.

This closes #3800


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

Branch: refs/heads/release-1.3
Commit: 3990d75aaedc8e03ef2facf5732c4a0fe52a7cdc
Parents: d88a62f
Author: Greg Hogan <co...@greghogan.com>
Authored: Thu Apr 27 14:43:18 2017 -0400
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 flink-clients/pom.xml                           |   6 +-
 flink-connectors/flink-avro/pom.xml             |   6 +-
 .../flink-connector-cassandra/pom.xml           |  12 +-
 .../flink-connector-elasticsearch-base/pom.xml  |  10 +-
 .../flink-connector-elasticsearch/pom.xml       |  12 +-
 .../flink-connector-elasticsearch2/pom.xml      |  12 +-
 .../flink-connector-elasticsearch5/pom.xml      |  12 +-
 .../flink-connector-filesystem/pom.xml          |  14 +--
 .../flink-connector-kafka-0.10/pom.xml          |  20 ++--
 .../flink-connector-kafka-0.8/pom.xml           |  18 +--
 .../flink-connector-kafka-0.9/pom.xml           |  18 +--
 .../flink-connector-kafka-base/pom.xml          |  16 +--
 .../flink-connector-kinesis/pom.xml             |  12 +-
 flink-connectors/flink-connector-nifi/pom.xml   |  10 +-
 .../flink-connector-rabbitmq/pom.xml            |   8 +-
 .../flink-connector-twitter/pom.xml             |   4 +-
 .../flink-hadoop-compatibility/pom.xml          |   6 +-
 flink-connectors/flink-hbase/pom.xml            |  18 +--
 flink-connectors/flink-hcatalog/pom.xml         |   2 +-
 flink-connectors/flink-jdbc/pom.xml             |   4 +-
 flink-contrib/flink-connector-wikiedits/pom.xml |   4 +-
 .../flink-statebackend-rocksdb/pom.xml          |  10 +-
 flink-contrib/flink-storm-examples/pom.xml      |  24 ++--
 flink-contrib/flink-storm/pom.xml               |   6 +-
 flink-contrib/flink-streaming-contrib/pom.xml   |  10 +-
 flink-contrib/flink-tweet-inputformat/pom.xml   |   6 +-
 flink-dist/pom.xml                              |  42 +++----
 flink-dist/src/main/assemblies/bin.xml          |   4 +-
 flink-dist/src/main/assemblies/opt.xml          |  24 ++--
 flink-examples/flink-examples-batch/pom.xml     |  22 ++--
 flink-examples/flink-examples-streaming/pom.xml |  36 +++---
 flink-examples/flink-examples-table/pom.xml     |  10 +-
 flink-examples/pom.xml                          |   4 +-
 flink-fs-tests/pom.xml                          |  12 +-
 flink-java8/pom.xml                             |  12 +-
 flink-libraries/flink-cep-scala/pom.xml         |  14 +--
 flink-libraries/flink-cep/pom.xml               |  12 +-
 flink-libraries/flink-gelly-examples/pom.xml    |  14 +--
 flink-libraries/flink-gelly-scala/pom.xml       |  12 +-
 flink-libraries/flink-gelly/pom.xml             |   8 +-
 flink-libraries/flink-ml/pom.xml                |   8 +-
 flink-libraries/flink-python/pom.xml            |   6 +-
 flink-libraries/flink-table/pom.xml             |  14 +--
 flink-mesos/pom.xml                             |  10 +-
 flink-metrics/flink-metrics-datadog/pom.xml     |   2 +-
 flink-metrics/flink-metrics-dropwizard/pom.xml  |   2 +-
 flink-metrics/flink-metrics-jmx/pom.xml         |   4 +-
 flink-metrics/flink-metrics-statsd/pom.xml      |   2 +-
 flink-optimizer/pom.xml                         |   4 +-
 .../main/resources/archetype-resources/pom.xml  |  30 ++---
 .../main/resources/archetype-resources/pom.xml  |  34 +++---
 flink-runtime-web/pom.xml                       |   8 +-
 flink-runtime/pom.xml                           |   2 +-
 flink-scala-shell/pom.xml                       |  14 +--
 flink-scala/pom.xml                             |   8 +-
 flink-streaming-java/pom.xml                    |   8 +-
 flink-streaming-scala/pom.xml                   |  14 +--
 .../flink-test-utils/pom.xml                    |   8 +-
 flink-tests/pom.xml                             |  36 +++---
 flink-yarn-tests/pom.xml                        |  12 +-
 flink-yarn/pom.xml                              |  10 +-
 tools/change-scala-version.sh                   | 117 -------------------
 tools/create_release_files.sh                   |  10 +-
 tools/deploy_to_maven.sh                        |   8 +-
 64 files changed, 376 insertions(+), 501 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-clients/pom.xml
----------------------------------------------------------------------
diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml
index 8dde2e9..deac7fc 100644
--- a/flink-clients/pom.xml
+++ b/flink-clients/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-clients_2.10</artifactId>
+	<artifactId>flink-clients_${scala.binary.version}</artifactId>
 	<name>flink-clients</name>
 
 	<packaging>jar</packaging>
@@ -47,13 +47,13 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-optimizer_2.10</artifactId>
+			<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/pom.xml b/flink-connectors/flink-avro/pom.xml
index 170b344..ed99eac 100644
--- a/flink-connectors/flink-avro/pom.xml
+++ b/flink-connectors/flink-avro/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-avro_2.10</artifactId>
+	<artifactId>flink-avro_${scala.binary.version}</artifactId>
 	<name>flink-avro</name>
 
 	<packaging>jar</packaging>
@@ -62,14 +62,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml
index d01c769..2722c30 100644
--- a/flink-connectors/flink-connector-cassandra/pom.xml
+++ b/flink-connectors/flink-connector-cassandra/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-cassandra_2.10</artifactId>
+	<artifactId>flink-connector-cassandra_${scala.binary.version}</artifactId>
 	<name>flink-connector-cassandra</name>
 
 	<packaging>jar</packaging>
@@ -94,7 +94,7 @@ under the License.
 	<dependencies>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -136,27 +136,27 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-elasticsearch-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
index e183354..79b4f50 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+	<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 	<name>flink-connector-elasticsearch-base</name>
 
 	<packaging>jar</packaging>
@@ -46,7 +46,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -61,14 +61,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -76,7 +76,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch/pom.xml b/flink-connectors/flink-connector-elasticsearch/pom.xml
index 07028df..93e4eb6 100644
--- a/flink-connectors/flink-connector-elasticsearch/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-elasticsearch_2.10</artifactId>
+	<artifactId>flink-connector-elasticsearch_${scala.binary.version}</artifactId>
 	<name>flink-connector-elasticsearch</name>
 
 	<packaging>jar</packaging>
@@ -46,14 +46,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+			<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -61,14 +61,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -76,7 +76,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+			<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-elasticsearch2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml
index ec1b7da..7e21b8f 100644
--- a/flink-connectors/flink-connector-elasticsearch2/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-elasticsearch2_2.10</artifactId>
+	<artifactId>flink-connector-elasticsearch2_${scala.binary.version}</artifactId>
 	<name>flink-connector-elasticsearch2</name>
 
 	<packaging>jar</packaging>
@@ -46,14 +46,14 @@ under the License.
  
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+			<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -68,14 +68,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -83,7 +83,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+			<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-elasticsearch5/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml
index 93d7bbe..72a0d18 100644
--- a/flink-connectors/flink-connector-elasticsearch5/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-elasticsearch5_2.10</artifactId>
+	<artifactId>flink-connector-elasticsearch5_${scala.binary.version}</artifactId>
 	<name>flink-connector-elasticsearch5</name>
 
 	<packaging>jar</packaging>
@@ -46,14 +46,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+			<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<!-- Elasticsearch Java Client has been moved to a different module in 5.x -->
@@ -88,14 +88,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -103,7 +103,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-elasticsearch-base_2.10</artifactId>
+			<artifactId>flink-connector-elasticsearch-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<exclusion>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml
index 28d9962..07b0ae1 100644
--- a/flink-connectors/flink-connector-filesystem/pom.xml
+++ b/flink-connectors/flink-connector-filesystem/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-filesystem_2.10</artifactId>
+	<artifactId>flink-connector-filesystem_${scala.binary.version}</artifactId>
 	<name>flink-connector-filesystem</name>
 
 	<packaging>jar</packaging>
@@ -45,7 +45,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -68,21 +68,21 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -90,7 +90,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -98,7 +98,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
index c17386d..231b22e 100644
--- a/flink-connectors/flink-connector-kafka-0.10/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-kafka-0.10_2.10</artifactId>
+	<artifactId>flink-connector-kafka-0.10_${scala.binary.version}</artifactId>
 	<name>flink-connector-kafka-0.10</name>
 
 	<packaging>jar</packaging>
@@ -46,7 +46,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
+			<artifactId>flink-connector-kafka-0.9_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<exclusion>
@@ -60,7 +60,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -75,7 +75,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project,
@@ -87,7 +87,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -95,7 +95,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
+			<artifactId>flink-connector-kafka-0.9_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<!-- exclude Kafka dependencies -->
@@ -110,7 +110,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<!-- exclude Kafka dependencies -->
@@ -133,7 +133,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -141,14 +141,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml
index ceccb3e..5e2ed2d 100644
--- a/flink-connectors/flink-connector-kafka-0.8/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-kafka-0.8_2.10</artifactId>
+	<artifactId>flink-connector-kafka-0.8_${scala.binary.version}</artifactId>
 	<name>flink-connector-kafka-0.8</name>
 
 	<packaging>jar</packaging>
@@ -46,7 +46,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -60,13 +60,13 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project,
@@ -122,7 +122,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -144,7 +144,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -152,7 +152,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -160,14 +160,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
index de4dedb..0140353 100644
--- a/flink-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-kafka-0.9_2.10</artifactId>
+	<artifactId>flink-connector-kafka-0.9_${scala.binary.version}</artifactId>
 	<name>flink-connector-kafka-0.9</name>
 
 	<packaging>jar</packaging>
@@ -46,14 +46,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<exclusion>
@@ -65,7 +65,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project,
@@ -83,7 +83,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -91,7 +91,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-base_2.10</artifactId>
+			<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<!-- exclude 0.8 dependencies -->
@@ -121,7 +121,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -129,14 +129,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml
index a6b2349..2cc94b0 100644
--- a/flink-connectors/flink-connector-kafka-base/pom.xml
+++ b/flink-connectors/flink-connector-kafka-base/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-kafka-base_2.10</artifactId>
+	<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
 	<name>flink-connector-kafka-base</name>
 
 	<packaging>jar</packaging>
@@ -46,14 +46,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project,
@@ -132,7 +132,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -140,14 +140,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -155,7 +155,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -163,7 +163,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_2.10</artifactId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml
index 7515f31..4628937 100644
--- a/flink-connectors/flink-connector-kinesis/pom.xml
+++ b/flink-connectors/flink-connector-kinesis/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-kinesis_2.10</artifactId>
+	<artifactId>flink-connector-kinesis_${scala.binary.version}</artifactId>
 	<name>flink-connector-kinesis</name>
 	<properties>
 		<aws.sdk.version>1.10.71</aws.sdk.version>
@@ -44,7 +44,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -57,7 +57,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -65,7 +65,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -73,7 +73,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -81,7 +81,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-nifi/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-nifi/pom.xml b/flink-connectors/flink-connector-nifi/pom.xml
index 67bfb29..efa7ae7 100644
--- a/flink-connectors/flink-connector-nifi/pom.xml
+++ b/flink-connectors/flink-connector-nifi/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-nifi_2.10</artifactId>
+	<artifactId>flink-connector-nifi_${scala.binary.version}</artifactId>
 	<name>flink-connector-nifi</name>
 
 	<packaging>jar</packaging>
@@ -48,26 +48,26 @@ under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-java_2.10</artifactId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
 			<scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-java_2.10</artifactId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
             <type>test-jar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-tests_2.10</artifactId>
+            <artifactId>flink-tests_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils_2.10</artifactId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
 			<type>test-jar</type>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-rabbitmq/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/pom.xml b/flink-connectors/flink-connector-rabbitmq/pom.xml
index f26e57c..38337f4 100644
--- a/flink-connectors/flink-connector-rabbitmq/pom.xml
+++ b/flink-connectors/flink-connector-rabbitmq/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-rabbitmq_2.10</artifactId>
+	<artifactId>flink-connector-rabbitmq_${scala.binary.version}</artifactId>
 	<name>flink-connector-rabbitmq</name>
 
 	<packaging>jar</packaging>
@@ -44,7 +44,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -57,7 +57,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -65,7 +65,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-connector-twitter/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-twitter/pom.xml b/flink-connectors/flink-connector-twitter/pom.xml
index c683da1..38bfb30 100644
--- a/flink-connectors/flink-connector-twitter/pom.xml
+++ b/flink-connectors/flink-connector-twitter/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-twitter_2.10</artifactId>
+	<artifactId>flink-connector-twitter_${scala.binary.version}</artifactId>
 	<name>flink-connector-twitter</name>
 
 	<packaging>jar</packaging>
@@ -43,7 +43,7 @@ under the License.
 	<dependencies>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-hadoop-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml
index 003225b..2dee17d 100644
--- a/flink-connectors/flink-hadoop-compatibility/pom.xml
+++ b/flink-connectors/flink-hadoop-compatibility/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+	<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
 	<name>flink-hadoop-compatibility</name>
 
 	<packaging>jar</packaging>
@@ -47,7 +47,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -70,7 +70,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml
index 6a295a8..949e27a 100644
--- a/flink-connectors/flink-hbase/pom.xml
+++ b/flink-connectors/flink-hbase/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-hbase_2.10</artifactId>
+	<artifactId>flink-hbase_${scala.binary.version}</artifactId>
 	<name>flink-hbase</name>
 	<packaging>jar</packaging>
 
@@ -89,7 +89,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -98,7 +98,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -107,7 +107,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project,
@@ -124,7 +124,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 
@@ -206,7 +206,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<exclusions>
@@ -219,13 +219,13 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<exclusions>
 				<exclusion>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-include-yarn_2.10</artifactId>
+					<artifactId>flink-shaded-include-yarn_${scala.binary.version}</artifactId>
 				</exclusion>
 			</exclusions>
 		</dependency>
@@ -271,7 +271,7 @@ under the License.
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml
index a975e53..a9fbceb 100644
--- a/flink-connectors/flink-hcatalog/pom.xml
+++ b/flink-connectors/flink-hcatalog/pom.xml
@@ -44,7 +44,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-connectors/flink-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/pom.xml b/flink-connectors/flink-jdbc/pom.xml
index 38a0163..a2bbaf4 100644
--- a/flink-connectors/flink-jdbc/pom.xml
+++ b/flink-connectors/flink-jdbc/pom.xml
@@ -39,7 +39,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project,
@@ -56,7 +56,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-contrib/flink-connector-wikiedits/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-connector-wikiedits/pom.xml b/flink-contrib/flink-connector-wikiedits/pom.xml
index fed8656..20b6db4 100644
--- a/flink-contrib/flink-connector-wikiedits/pom.xml
+++ b/flink-contrib/flink-connector-wikiedits/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-connector-wikiedits_2.10</artifactId>
+	<artifactId>flink-connector-wikiedits_${scala.binary.version}</artifactId>
 	<name>flink-connector-wikiedits</name>
 
 	<packaging>jar</packaging>
@@ -37,7 +37,7 @@ under the License.
 	<dependencies>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-contrib/flink-statebackend-rocksdb/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/pom.xml b/flink-contrib/flink-statebackend-rocksdb/pom.xml
index 2148706..527ca18 100644
--- a/flink-contrib/flink-statebackend-rocksdb/pom.xml
+++ b/flink-contrib/flink-statebackend-rocksdb/pom.xml
@@ -31,7 +31,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-statebackend-rocksdb_2.10</artifactId>
+	<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
 	<name>flink-statebackend-rocksdb</name>
 
 	<packaging>jar</packaging>
@@ -42,14 +42,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -78,7 +78,7 @@ under the License.
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -86,7 +86,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-contrib/flink-storm-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml
index 4d9d76c..6ef0f7b 100644
--- a/flink-contrib/flink-storm-examples/pom.xml
+++ b/flink-contrib/flink-storm-examples/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-storm-examples_2.10</artifactId>
+	<artifactId>flink-storm-examples_${scala.binary.version}</artifactId>
 	<name>flink-storm-examples</name>
 
 	<packaging>jar</packaging>
@@ -56,19 +56,19 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-storm_2.10</artifactId>
+			<artifactId>flink-storm_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-examples-batch_2.10</artifactId>
+			<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -95,14 +95,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -128,7 +128,7 @@ under the License.
 							<artifactItems>
 								<artifactItem>
 									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-examples-batch_2.10</artifactId>
+									<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 									<version>${project.version}</version>
 									<type>jar</type>
 									<overWrite>false</overWrite>
@@ -138,7 +138,7 @@ under the License.
 								</artifactItem>
 								<artifactItem>
 									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-storm_2.10</artifactId>
+									<artifactId>flink-storm_${scala.binary.version}</artifactId>
 									<version>${project.version}</version>
 									<type>jar</type>
 									<overWrite>false</overWrite>
@@ -326,8 +326,8 @@ under the License.
 									<!-- Storm's recursive dependencies -->
 									<include>org.yaml:snakeyaml</include>
 									<include>com.googlecode.json-simple:json-simple</include>
-									<include>org.apache.flink:flink-storm_2.10</include>
-									<include>org.apache.flink:flink-storm-examples_2.10</include>
+									<include>org.apache.flink:flink-storm_${scala.binary.version}</include>
+									<include>org.apache.flink:flink-storm-examples_${scala.binary.version}</include>
 								</includes>
 							</artifactSet>
 							<filters>
@@ -354,7 +354,7 @@ under the License.
 									</includes>
 								</filter>
 								<filter>
-									<artifact>org.apache.flink:flink-storm-examples_2.10</artifact>
+									<artifact>org.apache.flink:flink-storm-examples_${scala.binary.version}</artifact>
 									<includes>
 										<include>org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.class
 										</include>
@@ -366,7 +366,7 @@ under the License.
 									</includes>
 								</filter>
 								<filter>
-									<artifact>org.apache.flink:flink-storm_2.10</artifact>
+									<artifact>org.apache.flink:flink-storm_${scala.binary.version}</artifact>
 									<includes>
 										<include>org/apache/flink/storm/api/*.class</include>
 										<include>org/apache/flink/storm/util/*.class</include>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-contrib/flink-storm/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml
index 5189b3b..a10ff68 100644
--- a/flink-contrib/flink-storm/pom.xml
+++ b/flink-contrib/flink-storm/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-storm_2.10</artifactId>
+	<artifactId>flink-storm_${scala.binary.version}</artifactId>
 	<name>flink-storm</name>
 
 	<packaging>jar</packaging>
@@ -65,7 +65,7 @@ under the License.
 		<!-- Core streaming API -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -174,7 +174,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-contrib/flink-streaming-contrib/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-streaming-contrib/pom.xml b/flink-contrib/flink-streaming-contrib/pom.xml
index 2df0231..2381d70 100644
--- a/flink-contrib/flink-streaming-contrib/pom.xml
+++ b/flink-contrib/flink-streaming-contrib/pom.xml
@@ -31,7 +31,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-streaming-contrib_2.10</artifactId>
+	<artifactId>flink-streaming-contrib_${scala.binary.version}</artifactId>
 	<name>flink-streaming-contrib</name>
 
 	<packaging>jar</packaging>
@@ -42,19 +42,19 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -62,7 +62,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-contrib/flink-tweet-inputformat/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/pom.xml b/flink-contrib/flink-tweet-inputformat/pom.xml
index 5e0d79c..9328bb6 100644
--- a/flink-contrib/flink-tweet-inputformat/pom.xml
+++ b/flink-contrib/flink-tweet-inputformat/pom.xml
@@ -31,7 +31,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-tweet-inputformat_2.10</artifactId>
+	<artifactId>flink-tweet-inputformat_${scala.binary.version}</artifactId>
 	<name>flink-tweet-inputformat</name>
 
 	<packaging>jar</packaging>
@@ -44,12 +44,12 @@ under the License.
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 6d8debf..0493b98 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-dist_2.10</artifactId>
+	<artifactId>flink-dist_${scala.binary.version}</artifactId>
 	<name>flink-dist</name>
 	<packaging>jar</packaging>
 
@@ -51,61 +51,61 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime-web_2.10</artifactId>
+			<artifactId>flink-runtime-web_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-optimizer_2.10</artifactId>
+			<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_2.10</artifactId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-python_2.10</artifactId>
+			<artifactId>flink-python_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala-shell_2.10</artifactId>
+			<artifactId>flink-scala-shell_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -123,19 +123,19 @@ under the License.
         
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-mesos_2.10</artifactId>
+			<artifactId>flink-mesos_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-statebackend-rocksdb_2.10</artifactId>
+			<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-yarn_2.10</artifactId>
+			<artifactId>flink-yarn_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -163,7 +163,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-gelly-examples_2.10</artifactId>
+			<artifactId>flink-gelly-examples_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -214,42 +214,42 @@ under the License.
 		<!-- start optional Flink libraries -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-cep_2.10</artifactId>
+			<artifactId>flink-cep_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-cep-scala_2.10</artifactId>
+			<artifactId>flink-cep-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-gelly_2.10</artifactId>
+			<artifactId>flink-gelly_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-gelly-scala_2.10</artifactId>
+			<artifactId>flink-gelly-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-ml_2.10</artifactId>
+			<artifactId>flink-ml_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-dist/src/main/assemblies/bin.xml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml
index c3385e1..909beb4 100644
--- a/flink-dist/src/main/assemblies/bin.xml
+++ b/flink-dist/src/main/assemblies/bin.xml
@@ -39,7 +39,7 @@ under the License.
 			<useTransitiveFiltering>true</useTransitiveFiltering>
 
 			<includes>
-				<include>org.apache.flink:flink-python_2.10</include>
+				<include>org.apache.flink:flink-python_${scala.binary.version}</include>
 				<include>org.slf4j:slf4j-log4j12</include>
 				<include>log4j:log4j</include>
 			</includes>
@@ -49,7 +49,7 @@ under the License.
 	<files>
 		<!-- copy fat jar -->
 		<file>
-			<source>target/flink-dist_2.10-${project.version}.jar</source>
+			<source>target/flink-dist_${scala.binary.version}-${project.version}.jar</source>
 			<outputDirectory>lib/</outputDirectory>
 			<fileMode>0644</fileMode>
 		</file>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-dist/src/main/assemblies/opt.xml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml
index 0386b92..23846be 100644
--- a/flink-dist/src/main/assemblies/opt.xml
+++ b/flink-dist/src/main/assemblies/opt.xml
@@ -31,47 +31,47 @@
 	<files>
 		<!-- CEP -->
 		<file>
-			<source>../flink-libraries/flink-cep/target/flink-cep_2.10-${project.version}.jar</source>
+			<source>../flink-libraries/flink-cep/target/flink-cep_${scala.binary.version}-${project.version}.jar</source>
 			<outputDirectory>opt/</outputDirectory>
-			<destName>flink-cep_2.10-${project.version}.jar</destName>
+			<destName>flink-cep_${scala.binary.version}-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
 
 		<file>
-			<source>../flink-libraries/flink-cep-scala/target/flink-cep-scala_2.10-${project.version}.jar</source>
+			<source>../flink-libraries/flink-cep-scala/target/flink-cep-scala_${scala.binary.version}-${project.version}.jar</source>
 			<outputDirectory>opt/</outputDirectory>
-			<destName>flink-cep-scala_2.10-${project.version}.jar</destName>
+			<destName>flink-cep-scala_${scala.binary.version}-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
 
 		<!-- Gelly -->
 		<file>
-			<source>../flink-libraries/flink-gelly/target/flink-gelly_2.10-${project.version}.jar</source>
+			<source>../flink-libraries/flink-gelly/target/flink-gelly_${scala.binary.version}-${project.version}.jar</source>
 			<outputDirectory>opt/</outputDirectory>
-			<destName>flink-gelly_2.10-${project.version}.jar</destName>
+			<destName>flink-gelly_${scala.binary.version}-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
 
 		<file>
-			<source>../flink-libraries/flink-gelly-scala/target/flink-gelly-scala_2.10-${project.version}-jar-with-dependencies.jar</source>
+			<source>../flink-libraries/flink-gelly-scala/target/flink-gelly-scala_${scala.binary.version}-${project.version}-jar-with-dependencies.jar</source>
 			<outputDirectory>opt/</outputDirectory>
-			<destName>flink-gelly-scala_2.10-${project.version}.jar</destName>
+			<destName>flink-gelly-scala_${scala.binary.version}-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
 
 		<!-- TableAPI-->
 		<file>
-			<source>../flink-libraries/flink-table/target/flink-table_2.10-${project.version}.jar</source>
+			<source>../flink-libraries/flink-table/target/flink-table_${scala.binary.version}-${project.version}.jar</source>
 			<outputDirectory>opt/</outputDirectory>
-			<destName>flink-table_2.10-${project.version}.jar</destName>
+			<destName>flink-table_${scala.binary.version}-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
 
 		<!-- ML -->
 		<file>
-			<source>../flink-libraries/flink-ml/target/flink-ml_2.10-${project.version}-jar-with-dependencies.jar</source>
+			<source>../flink-libraries/flink-ml/target/flink-ml_${scala.binary.version}-${project.version}-jar-with-dependencies.jar</source>
 			<outputDirectory>opt/</outputDirectory>
-			<destName>flink-ml_2.10-${project.version}.jar</destName>
+			<destName>flink-ml_${scala.binary.version}-${project.version}.jar</destName>
 			<fileMode>0644</fileMode>
 		</file>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-examples/flink-examples-batch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-batch/pom.xml b/flink-examples/flink-examples-batch/pom.xml
index 269f5a6..cbff7e2 100644
--- a/flink-examples/flink-examples-batch/pom.xml
+++ b/flink-examples/flink-examples-batch/pom.xml
@@ -23,12 +23,12 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-examples_2.10</artifactId>
+		<artifactId>flink-examples_${scala.binary.version}</artifactId>
 		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-examples-batch_2.10</artifactId>
+	<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 	<name>flink-examples-batch</name>
 	<packaging>jar</packaging>
 
@@ -41,7 +41,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 	</dependencies>
@@ -363,14 +363,14 @@ under the License.
 						</goals>
 						<configuration> 
 							<target>
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-KMeans.jar" tofile="${project.basedir}/target/KMeans.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-ConnectedComponents.jar" tofile="${project.basedir}/target/ConnectedComponents.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-EnumTriangles.jar" tofile="${project.basedir}/target/EnumTriangles.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-PageRank.jar" tofile="${project.basedir}/target/PageRank.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-TransitiveClosure.jar" tofile="${project.basedir}/target/TransitiveClosure.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-WebLogAnalysis.jar" tofile="${project.basedir}/target/WebLogAnalysis.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-WordCount.jar" tofile="${project.basedir}/target/WordCount.jar" />
-								<copy file="${project.basedir}/target/flink-examples-batch_2.10-${project.version}-DistCp.jar" tofile="${project.basedir}/target/DistCp.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-KMeans.jar" tofile="${project.basedir}/target/KMeans.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-ConnectedComponents.jar" tofile="${project.basedir}/target/ConnectedComponents.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-EnumTriangles.jar" tofile="${project.basedir}/target/EnumTriangles.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-PageRank.jar" tofile="${project.basedir}/target/PageRank.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-TransitiveClosure.jar" tofile="${project.basedir}/target/TransitiveClosure.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-WebLogAnalysis.jar" tofile="${project.basedir}/target/WebLogAnalysis.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-WordCount.jar" tofile="${project.basedir}/target/WordCount.jar" />
+								<copy file="${project.basedir}/target/flink-examples-batch_${scala.binary.version}-${project.version}-DistCp.jar" tofile="${project.basedir}/target/DistCp.jar" />
 							</target>
 						</configuration>
 					</execution>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-examples/flink-examples-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index 2bacf25..b52116b 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -24,12 +24,12 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-examples_2.10</artifactId>
+		<artifactId>flink-examples_${scala.binary.version}</artifactId>
 		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-examples-streaming_2.10</artifactId>
+	<artifactId>flink-examples-streaming_${scala.binary.version}</artifactId>
 	<name>flink-examples-streaming</name>
 
 	<packaging>jar</packaging>
@@ -40,31 +40,31 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-examples-batch_2.10</artifactId>
+			<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-twitter_2.10</artifactId>
+			<artifactId>flink-connector-twitter_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-kafka-0.8_2.10</artifactId>
+			<artifactId>flink-connector-kafka-0.8_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -72,14 +72,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -116,7 +116,7 @@ under the License.
 								<!-- For WordCount example data -->
 								<artifactItem>
 									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-examples-batch_2.10</artifactId>
+									<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 									<version>${project.version}</version>
 									<type>jar</type>
 									<overWrite>false</overWrite>
@@ -126,7 +126,7 @@ under the License.
 								<!-- For JSON utilities -->
 								<artifactItem>
 									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-connector-twitter_2.10</artifactId>
+									<artifactId>flink-connector-twitter_${scala.binary.version}</artifactId>
 									<version>${project.version}</version>
 									<type>jar</type>
 									<overWrite>false</overWrite>
@@ -475,13 +475,13 @@ under the License.
 						</goals>
 						<configuration>
 							<target>
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-IncrementalLearning.jar" tofile="${project.basedir}/target/IncrementalLearning.jar" />
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-Iteration.jar" tofile="${project.basedir}/target/Iteration.jar" />
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-SessionWindowing.jar" tofile="${project.basedir}/target/SessionWindowing.jar" />
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-TopSpeedWindowing.jar" tofile="${project.basedir}/target/TopSpeedWindowing.jar" />
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-WindowJoin.jar" tofile="${project.basedir}/target/WindowJoin.jar" />
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-WordCount.jar" tofile="${project.basedir}/target/WordCount.jar" />
-								<copy file="${project.basedir}/target/flink-examples-streaming_2.10-${project.version}-SocketWindowWordCount.jar" tofile="${project.basedir}/target/SocketWindowWordCount.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-IncrementalLearning.jar" tofile="${project.basedir}/target/IncrementalLearning.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-Iteration.jar" tofile="${project.basedir}/target/Iteration.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-SessionWindowing.jar" tofile="${project.basedir}/target/SessionWindowing.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-TopSpeedWindowing.jar" tofile="${project.basedir}/target/TopSpeedWindowing.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-WindowJoin.jar" tofile="${project.basedir}/target/WindowJoin.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-WordCount.jar" tofile="${project.basedir}/target/WordCount.jar" />
+								<copy file="${project.basedir}/target/flink-examples-streaming_${scala.binary.version}-${project.version}-SocketWindowWordCount.jar" tofile="${project.basedir}/target/SocketWindowWordCount.jar" />
 							</target>
 						</configuration>
 					</execution>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-examples/flink-examples-table/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml
index ce1c701..e59d8c6 100644
--- a/flink-examples/flink-examples-table/pom.xml
+++ b/flink-examples/flink-examples-table/pom.xml
@@ -24,27 +24,27 @@ under the License.
 
 	<parent>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-examples_2.10</artifactId>
+		<artifactId>flink-examples_${scala.binary.version}</artifactId>
 		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 
 	<name>flink-examples-table</name>
-	<artifactId>flink-examples-table_2.10</artifactId>
+	<artifactId>flink-examples-table_${scala.binary.version}</artifactId>
 	<packaging>jar</packaging>
 
 	<dependencies>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table_2.10</artifactId>
+			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -78,7 +78,7 @@ under the License.
 				<artifactId>maven-shade-plugin</artifactId>
 				<executions>
 					<execution>
-						<id>flink-table-examples_2.10</id>
+						<id>flink-table-examples_${scala.binary.version}</id>
 						<phase>package</phase>
 						<goals>
 							<goal>shade</goal>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml
index 409c8f1..644d9eb 100644
--- a/flink-examples/pom.xml
+++ b/flink-examples/pom.xml
@@ -28,7 +28,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-examples_2.10</artifactId>
+	<artifactId>flink-examples_${scala.binary.version}</artifactId>
 	<name>flink-examples</name>
 	<packaging>pom</packaging>
 
@@ -44,7 +44,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-fs-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml
index c354c80..446d167 100644
--- a/flink-fs-tests/pom.xml
+++ b/flink-fs-tests/pom.xml
@@ -27,7 +27,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-fs-tests_2.10</artifactId>
+	<artifactId>flink-fs-tests_${scala.binary.version}</artifactId>
 	<name>flink-fs-tests</name>
 
 	<packaging>jar</packaging>
@@ -45,7 +45,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -53,28 +53,28 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-examples-batch_2.10</artifactId>
+			<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_2.10</artifactId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-java8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml
index 8de3b92..fb7da02 100644
--- a/flink-java8/pom.xml
+++ b/flink-java8/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-java8_2.10</artifactId>
+	<artifactId>flink-java8_${scala.binary.version}</artifactId>
 	<name>flink-java8</name>
 
 	<packaging>jar</packaging>
@@ -51,13 +51,13 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-examples-batch_2.10</artifactId>
+			<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -65,14 +65,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-cep_2.10</artifactId>
+			<artifactId>flink-cep_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
@@ -116,7 +116,7 @@ under the License.
 							<artifactItems>
 								<artifactItem>
 									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-examples-batch_2.10</artifactId>
+									<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 									<version>${project.version}</version>
 									<type>jar</type>
 									<overWrite>false</overWrite>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-cep-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/pom.xml b/flink-libraries/flink-cep-scala/pom.xml
index ca5bbce..c9b4a46 100644
--- a/flink-libraries/flink-cep-scala/pom.xml
+++ b/flink-libraries/flink-cep-scala/pom.xml
@@ -30,7 +30,7 @@ under the License.
         <relativePath>..</relativePath>
     </parent>
     
-    <artifactId>flink-cep-scala_2.10</artifactId>
+    <artifactId>flink-cep-scala_${scala.binary.version}</artifactId>
     <name>flink-cep-scala</name>
     <packaging>jar</packaging>
 
@@ -40,13 +40,13 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-cep_2.10</artifactId>
+            <artifactId>flink-cep_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-scala_2.10</artifactId>
+            <artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>
@@ -83,14 +83,14 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils_2.10</artifactId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-tests_2.10</artifactId>
+            <artifactId>flink-tests_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
             <type>test-jar</type>
@@ -98,7 +98,7 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-scala_2.10</artifactId>
+            <artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
             <type>test-jar</type>
@@ -106,7 +106,7 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-cep_2.10</artifactId>
+            <artifactId>flink-cep_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
             <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-cep/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml
index 7f89c24..35045c0 100644
--- a/flink-libraries/flink-cep/pom.xml
+++ b/flink-libraries/flink-cep/pom.xml
@@ -30,7 +30,7 @@ under the License.
         <relativePath>..</relativePath>
     </parent>
 
-    <artifactId>flink-cep_2.10</artifactId>
+    <artifactId>flink-cep_${scala.binary.version}</artifactId>
     <name>flink-cep</name>
     <packaging>jar</packaging>
 
@@ -47,7 +47,7 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-java_2.10</artifactId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>
@@ -62,14 +62,14 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils_2.10</artifactId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
         
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-java_2.10</artifactId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <type>test-jar</type>
             <scope>test</scope>
@@ -77,7 +77,7 @@ under the License.
 
        <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-runtime_2.10</artifactId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <type>test-jar</type>
             <scope>test</scope>
@@ -85,7 +85,7 @@ under the License.
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-statebackend-rocksdb_2.10</artifactId>
+            <artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>


[14/14] flink git commit: [FLINK-6504] [checkpoint] Fix synchronization on materializedSstFiles in RocksDBKeyedStateBackend

Posted by sr...@apache.org.
[FLINK-6504] [checkpoint] Fix synchronization on materializedSstFiles in RocksDBKeyedStateBackend


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

Branch: refs/heads/release-1.3
Commit: f7d79d8fd9e6b2691c75af1214666cc99b5aaca7
Parents: b30b8ee
Author: Stefan Richter <s....@data-artisans.com>
Authored: Thu May 11 11:59:47 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../contrib/streaming/state/RocksDBKeyedStateBackend.java     | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f7d79d8f/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 1080e59..b9468f7 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -828,9 +828,12 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 
 		void takeSnapshot() throws Exception {
+			assert (Thread.holdsLock(stateBackend.asyncSnapshotLock));
+
 			// use the last completed checkpoint as the comparison base.
 			baseSstFiles = stateBackend.materializedSstFiles.get(stateBackend.lastCompletedCheckpointId);
 
+
 			// save meta data
 			for (Map.Entry<String, Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>> stateMetaInfoEntry
 					: stateBackend.kvStateInformation.entrySet()) {
@@ -888,7 +891,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			sstFiles.putAll(newSstFiles);
 			sstFiles.putAll(oldSstFiles);
 
-			stateBackend.materializedSstFiles.put(checkpointId, sstFiles);
+			synchronized (stateBackend.asyncSnapshotLock) {
+				stateBackend.materializedSstFiles.put(checkpointId, sstFiles);
+			}
 
 			return new RocksDBIncrementalKeyedStateHandle(
 				stateBackend.operatorIdentifier,


[12/14] flink git commit: [FLINK-6545] [checkpoint] Make incremental checkpoints externalizable

Posted by sr...@apache.org.
[FLINK-6545] [checkpoint] Make incremental checkpoints externalizable


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

Branch: refs/heads/release-1.3
Commit: 7c6f348575ad4baf21279e184d565f0a431ccca0
Parents: 37e3dea
Author: Stefan Richter <s....@data-artisans.com>
Authored: Thu May 11 21:04:29 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../RocksDBIncrementalKeyedStateHandle.java     | 241 --------------
 .../state/RocksDBKeyedStateBackend.java         | 155 ++++-----
 .../org/apache/flink/util/StringBasedID.java    |  69 ++++
 .../savepoint/SavepointV2Serializer.java        |  83 ++++-
 .../state/IncrementalKeyedStateHandle.java      | 324 +++++++++++++++++++
 .../state/PlaceholderStreamStateHandle.java     |  88 +++++
 .../flink/runtime/state/SharedStateHandle.java  |  39 ---
 .../runtime/state/SharedStateRegistry.java      |   2 +-
 .../runtime/state/SharedStateRegistryKey.java   |  42 +--
 .../apache/flink/runtime/state/StateHandle.java |  37 ---
 .../flink/runtime/state/StateHandleID.java      |  37 +++
 .../savepoint/CheckpointTestUtils.java          |  87 +++--
 .../savepoint/SavepointV2SerializerTest.java    |   1 -
 .../flink/runtime/state/StateUtilTest.java      |  36 ---
 14 files changed, 736 insertions(+), 505 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
deleted file mode 100644
index 961182d..0000000
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.contrib.streaming.state;
-
-import org.apache.flink.runtime.state.CompositeStateHandle;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyedStateHandle;
-import org.apache.flink.runtime.state.SharedStateRegistry;
-import org.apache.flink.runtime.state.SharedStateRegistryKey;
-import org.apache.flink.runtime.state.StateUtil;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-/**
- * The handle to states in incremental snapshots taken by {@link RocksDBKeyedStateBackend}.
- *
- * The states contained in an incremental snapshot include
- * <ul>
- * <li> New SST state which includes the sst files produced since the last completed
- *   checkpoint. These files can be referenced by succeeding checkpoints if the
- *   checkpoint succeeds to complete. </li>
- * <li> Old SST state which includes the sst files materialized in previous
- *   checkpoints. </li>
- * <li> MISC state which include the other files in the RocksDB instance, e.g. the
- *   LOG and MANIFEST files. These files are mutable, hence cannot be shared by
- *   other checkpoints. </li>
- * <li> Meta state which includes the information of existing states. </li>
- * </ul>
- */
-public class RocksDBIncrementalKeyedStateHandle implements KeyedStateHandle, CompositeStateHandle {
-
-	private static final Logger LOG = LoggerFactory.getLogger(RocksDBIncrementalKeyedStateHandle.class);
-
-	private static final long serialVersionUID = -8328808513197388231L;
-
-	private final String operatorIdentifier;
-
-	private final KeyGroupRange keyGroupRange;
-
-	private final long checkpointId;
-
-	private final Map<String, StreamStateHandle> unregisteredSstFiles;
-
-	private final Map<String, StreamStateHandle> registeredSstFiles;
-
-	private final Map<String, StreamStateHandle> miscFiles;
-
-	private final StreamStateHandle metaStateHandle;
-
-	/**
-	 * True if the state handle has already registered shared states.
-	 *
-	 * Once the shared states are registered, it's the {@link SharedStateRegistry}'s
-	 * responsibility to maintain the shared states. But in the cases where the
-	 * state handle is discarded before performing the registration, the handle
-	 * should delete all the shared states created by it.
-	 */
-	private boolean registered;
-
-	RocksDBIncrementalKeyedStateHandle(
-			String operatorIdentifier,
-			KeyGroupRange keyGroupRange,
-			long checkpointId,
-			Map<String, StreamStateHandle> unregisteredSstFiles,
-			Map<String, StreamStateHandle> registeredSstFiles,
-			Map<String, StreamStateHandle> miscFiles,
-			StreamStateHandle metaStateHandle) {
-
-		this.operatorIdentifier = Preconditions.checkNotNull(operatorIdentifier);
-		this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
-		this.checkpointId = checkpointId;
-		this.unregisteredSstFiles = Preconditions.checkNotNull(unregisteredSstFiles);
-		this.registeredSstFiles = Preconditions.checkNotNull(registeredSstFiles);
-		this.miscFiles = Preconditions.checkNotNull(miscFiles);
-		this.metaStateHandle = Preconditions.checkNotNull(metaStateHandle);
-		this.registered = false;
-	}
-
-	@Override
-	public KeyGroupRange getKeyGroupRange() {
-		return keyGroupRange;
-	}
-
-	long getCheckpointId() {
-		return checkpointId;
-	}
-
-	Map<String, StreamStateHandle> getUnregisteredSstFiles() {
-		return unregisteredSstFiles;
-	}
-
-	Map<String, StreamStateHandle> getRegisteredSstFiles() {
-		return registeredSstFiles;
-	}
-
-	Map<String, StreamStateHandle> getMiscFiles() {
-		return miscFiles;
-	}
-
-	StreamStateHandle getMetaStateHandle() {
-		return metaStateHandle;
-	}
-
-	@Override
-	public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
-		if (this.keyGroupRange.getIntersection(keyGroupRange) != KeyGroupRange.EMPTY_KEY_GROUP_RANGE) {
-			return this;
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public void discardState() throws Exception {
-
-		Preconditions.checkState(!registered, "Attempt to dispose a registered composite state with registered shared state. Must unregister first.");
-
-		try {
-			metaStateHandle.discardState();
-		} catch (Exception e) {
-			LOG.warn("Could not properly discard meta data.", e);
-		}
-
-		try {
-			StateUtil.bestEffortDiscardAllStateObjects(miscFiles.values());
-		} catch (Exception e) {
-			LOG.warn("Could not properly discard misc file states.", e);
-		}
-
-		try {
-			StateUtil.bestEffortDiscardAllStateObjects(unregisteredSstFiles.values());
-		} catch (Exception e) {
-			LOG.warn("Could not properly discard new sst file states.", e);
-		}
-
-	}
-
-	@Override
-	public long getStateSize() {
-		long size = StateUtil.getStateSize(metaStateHandle);
-
-		for (StreamStateHandle newSstFileHandle : unregisteredSstFiles.values()) {
-			size += newSstFileHandle.getStateSize();
-		}
-
-		for (StreamStateHandle oldSstFileHandle : registeredSstFiles.values()) {
-			size += oldSstFileHandle.getStateSize();
-		}
-
-		for (StreamStateHandle miscFileHandle : miscFiles.values()) {
-			size += miscFileHandle.getStateSize();
-		}
-
-		return size;
-	}
-
-	@Override
-	public void registerSharedStates(SharedStateRegistry stateRegistry) {
-
-		Preconditions.checkState(!registered, "The state handle has already registered its shared states.");
-
-		for (Map.Entry<String, StreamStateHandle> newSstFileEntry : unregisteredSstFiles.entrySet()) {
-			SharedStateRegistryKey registryKey =
-				createSharedStateRegistryKeyFromFileName(newSstFileEntry.getKey());
-
-			SharedStateRegistry.Result result =
-				stateRegistry.registerNewReference(registryKey, newSstFileEntry.getValue());
-
-			// We update our reference with the result from the registry, to prevent the following
-			// problem:
-			// A previous checkpoint n has already registered the state. This can happen if a
-			// following checkpoint (n + x) wants to reference the same state before the backend got
-			// notified that checkpoint n completed. In this case, the shared registry did
-			// deduplication and returns the previous reference.
-			newSstFileEntry.setValue(result.getReference());
-		}
-
-		for (Map.Entry<String, StreamStateHandle> oldSstFileName : registeredSstFiles.entrySet()) {
-			SharedStateRegistryKey registryKey =
-				createSharedStateRegistryKeyFromFileName(oldSstFileName.getKey());
-
-			SharedStateRegistry.Result result = stateRegistry.obtainReference(registryKey);
-
-			// Again we update our state handle with the result from the registry, thus replacing
-			// placeholder state handles with the originals.
-			oldSstFileName.setValue(result.getReference());
-		}
-
-		// Migrate state from unregistered to registered, so that it will not count as private state
-		// for #discardState() from now.
-		registeredSstFiles.putAll(unregisteredSstFiles);
-		unregisteredSstFiles.clear();
-
-		registered = true;
-	}
-
-	@Override
-	public void unregisterSharedStates(SharedStateRegistry stateRegistry) {
-
-		Preconditions.checkState(registered, "The state handle has not registered its shared states yet.");
-
-		for (Map.Entry<String, StreamStateHandle> newSstFileEntry : unregisteredSstFiles.entrySet()) {
-			SharedStateRegistryKey registryKey =
-				createSharedStateRegistryKeyFromFileName(newSstFileEntry.getKey());
-			stateRegistry.releaseReference(registryKey);
-		}
-
-		for (Map.Entry<String, StreamStateHandle> oldSstFileEntry : registeredSstFiles.entrySet()) {
-			SharedStateRegistryKey registryKey =
-				createSharedStateRegistryKeyFromFileName(oldSstFileEntry.getKey());
-			stateRegistry.releaseReference(registryKey);
-		}
-
-		registered = false;
-	}
-
-	private SharedStateRegistryKey createSharedStateRegistryKeyFromFileName(String fileName) {
-		return new SharedStateRegistryKey(operatorIdentifier + "-" + keyGroupRange + "-" + fileName);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index b9468f7..4bd94fd 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -55,13 +55,16 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.DoneFuture;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
 import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
 import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
+import org.apache.flink.runtime.state.StateHandleID;
 import org.apache.flink.runtime.state.StateMigrationUtil;
 import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.StateUtil;
@@ -72,7 +75,6 @@ import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.runtime.state.internal.InternalReducingState;
 import org.apache.flink.runtime.state.internal.InternalValueState;
-import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
 import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.IOUtils;
@@ -172,7 +174,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	private final boolean enableIncrementalCheckpointing;
 
 	/** The sst files materialized in pending checkpoints */
-	private final SortedMap<Long, Map<String, StreamStateHandle>> materializedSstFiles = new TreeMap<>();
+	private final SortedMap<Long, Map<StateHandleID, StreamStateHandle>> materializedSstFiles = new TreeMap<>();
 
 	/** The identifier of the last completed checkpoint */
 	private long lastCompletedCheckpointId = -1;
@@ -723,7 +725,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private final long checkpointTimestamp;
 
 		/** All sst files that were part of the last previously completed checkpoint */
-		private Map<String, StreamStateHandle> baseSstFiles;
+		private Map<StateHandleID, StreamStateHandle> baseSstFiles;
 
 		/** The state meta data */
 		private final List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> stateMetaInfoSnapshots = new ArrayList<>();
@@ -735,13 +737,13 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private final CloseableRegistry closeableRegistry = new CloseableRegistry();
 
 		// new sst files since the last completed checkpoint
-		private final Map<String, StreamStateHandle> newSstFiles = new HashMap<>();
+		private final Map<StateHandleID, StreamStateHandle> newSstFiles = new HashMap<>();
 
 		// old sst files which have been materialized in previous completed checkpoints
-		private final Map<String, StreamStateHandle> oldSstFiles = new HashMap<>();
+		private final Map<StateHandleID, StreamStateHandle> oldSstFiles = new HashMap<>();
 
 		// handles to the misc files in the current snapshot
-		private final Map<String, StreamStateHandle> miscFiles = new HashMap<>();
+		private final Map<StateHandleID, StreamStateHandle> miscFiles = new HashMap<>();
 
 		private StreamStateHandle metaStateHandle = null;
 
@@ -865,8 +867,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			FileStatus[] fileStatuses = backupFileSystem.listStatus(backupPath);
 			if (fileStatuses != null) {
 				for (FileStatus fileStatus : fileStatuses) {
-					Path filePath = fileStatus.getPath();
-					String fileName = filePath.getName();
+					final Path filePath = fileStatus.getPath();
+					final String fileName = filePath.getName();
+					final StateHandleID stateHandleID = new StateHandleID(fileName);
 
 					if (fileName.endsWith(SST_FILE_SUFFIX)) {
 						StreamStateHandle fileHandle =
@@ -874,20 +877,24 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 						if (fileHandle == null) {
 							fileHandle = materializeStateData(filePath);
-							newSstFiles.put(fileName, fileHandle);
+							newSstFiles.put(stateHandleID, fileHandle);
 						} else {
 							// we introduce a placeholder state handle, that is replaced with the
 							// original from the shared state registry (created from a previous checkpoint)
-							oldSstFiles.put(fileName, new PlaceholderStreamStateHandle(fileHandle.getStateSize()));
+							oldSstFiles.put(
+								stateHandleID,
+								new PlaceholderStreamStateHandle(fileHandle.getStateSize()));
 						}
 					} else {
 						StreamStateHandle fileHandle = materializeStateData(filePath);
-						miscFiles.put(fileName, fileHandle);
+						miscFiles.put(stateHandleID, fileHandle);
 					}
 				}
 			}
 
-			Map<String, StreamStateHandle> sstFiles = new HashMap<>(newSstFiles.size() + oldSstFiles.size());
+			Map<StateHandleID, StreamStateHandle> sstFiles =
+				new HashMap<>(newSstFiles.size() + oldSstFiles.size());
+
 			sstFiles.putAll(newSstFiles);
 			sstFiles.putAll(oldSstFiles);
 
@@ -895,7 +902,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				stateBackend.materializedSstFiles.put(checkpointId, sstFiles);
 			}
 
-			return new RocksDBIncrementalKeyedStateHandle(
+			return new IncrementalKeyedStateHandle(
 				stateBackend.operatorIdentifier,
 				stateBackend.keyGroupRange,
 				checkpointId,
@@ -940,39 +947,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				}
 			}
 		}
-
-		/**
-		 * A placeholder state handle for shared state that will replaced by an original that was
-		 * created in a previous checkpoint. So we don't have to send the handle twice, e.g. in
-		 * case of {@link ByteStreamStateHandle}.
-		 */
-		private static final class PlaceholderStreamStateHandle implements StreamStateHandle {
-
-			private static final long serialVersionUID = 1L;
-
-			/** We remember the size of the original file for which this is a placeholder */
-			private final long originalSize;
-
-			public PlaceholderStreamStateHandle(long originalSize) {
-				this.originalSize = originalSize;
-			}
-
-			@Override
-			public FSDataInputStream openInputStream() {
-				throw new UnsupportedOperationException(
-					"This is only a placeholder to be replaced by a real StreamStateHandle in the checkpoint coordinator.");
-			}
-
-			@Override
-			public void discardState() throws Exception {
-				// nothing to do.
-			}
-
-			@Override
-			public long getStateSize() {
-				return originalSize;
-			}
-		}
 	}
 
 	@Override
@@ -989,7 +963,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			} else if (MigrationUtil.isOldSavepointKeyedState(restoreState)) {
 				LOG.info("Converting RocksDB state from old savepoint.");
 				restoreOldSavepointKeyedState(restoreState);
-			} else if (restoreState.iterator().next() instanceof RocksDBIncrementalKeyedStateHandle) {
+			} else if (restoreState.iterator().next() instanceof IncrementalKeyedStateHandle) {
 				RocksDBIncrementalRestoreOperation restoreOperation = new RocksDBIncrementalRestoreOperation(this);
 				restoreOperation.restore(restoreState);
 			} else {
@@ -1302,7 +1276,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 
 		private void restoreInstance(
-				RocksDBIncrementalKeyedStateHandle restoreStateHandle,
+				IncrementalKeyedStateHandle restoreStateHandle,
 				boolean hasExtraKeys) throws Exception {
 
 			// read state data
@@ -1311,29 +1285,16 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				UUID.randomUUID().toString());
 
 			try {
-				Map<String, StreamStateHandle> newSstFiles = restoreStateHandle.getUnregisteredSstFiles();
-				for (Map.Entry<String, StreamStateHandle> newSstFileEntry : newSstFiles.entrySet()) {
-					String fileName = newSstFileEntry.getKey();
-					StreamStateHandle remoteFileHandle = newSstFileEntry.getValue();
+				final Map<StateHandleID, StreamStateHandle> newSstFiles =
+					restoreStateHandle.getCreatedSharedState();
+				final Map<StateHandleID, StreamStateHandle> oldSstFiles =
+					restoreStateHandle.getReferencedSharedState();
+				final Map<StateHandleID, StreamStateHandle> miscFiles =
+					restoreStateHandle.getPrivateState();
 
-					readStateData(new Path(restoreInstancePath, fileName), remoteFileHandle);
-				}
-
-				Map<String, StreamStateHandle> oldSstFiles = restoreStateHandle.getRegisteredSstFiles();
-				for (Map.Entry<String, StreamStateHandle> oldSstFileEntry : oldSstFiles.entrySet()) {
-					String fileName = oldSstFileEntry.getKey();
-					StreamStateHandle remoteFileHandle = oldSstFileEntry.getValue();
-
-					readStateData(new Path(restoreInstancePath, fileName), remoteFileHandle);
-				}
-
-				Map<String, StreamStateHandle> miscFiles = restoreStateHandle.getMiscFiles();
-				for (Map.Entry<String, StreamStateHandle> miscFileEntry : miscFiles.entrySet()) {
-					String fileName = miscFileEntry.getKey();
-					StreamStateHandle remoteFileHandle = miscFileEntry.getValue();
-
-					readStateData(new Path(restoreInstancePath, fileName), remoteFileHandle);
-				}
+				readAllStateData(newSstFiles, restoreInstancePath);
+				readAllStateData(oldSstFiles, restoreInstancePath);
+				readAllStateData(miscFiles, restoreInstancePath);
 
 				// read meta data
 				List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> stateMetaInfoSnapshots =
@@ -1425,26 +1386,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 						throw new IOException("Could not create RocksDB data directory.");
 					}
 
-					for (String newSstFileName : newSstFiles.keySet()) {
-						File restoreFile = new File(restoreInstancePath.getPath(), newSstFileName);
-						File targetFile = new File(stateBackend.instanceRocksDBPath, newSstFileName);
-
-						Files.createLink(targetFile.toPath(), restoreFile.toPath());
-					}
-
-					for (String oldSstFileName : oldSstFiles.keySet()) {
-						File restoreFile = new File(restoreInstancePath.getPath(), oldSstFileName);
-						File targetFile = new File(stateBackend.instanceRocksDBPath, oldSstFileName);
-
-						Files.createLink(targetFile.toPath(), restoreFile.toPath());
-					}
-
-					for (String miscFileName : miscFiles.keySet()) {
-						File restoreFile = new File(restoreInstancePath.getPath(), miscFileName);
-						File targetFile = new File(stateBackend.instanceRocksDBPath, miscFileName);
-
-						Files.createLink(targetFile.toPath(), restoreFile.toPath());
-					}
+					createFileHardLinksInRestorePath(newSstFiles, restoreInstancePath);
+					createFileHardLinksInRestorePath(oldSstFiles, restoreInstancePath);
+					createFileHardLinksInRestorePath(miscFiles, restoreInstancePath);
 
 					List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
 					stateBackend.db = stateBackend.openDB(
@@ -1470,7 +1414,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 
 					// use the restore sst files as the base for succeeding checkpoints
-					Map<String, StreamStateHandle> sstFiles = new HashMap<>();
+					Map<StateHandleID, StreamStateHandle> sstFiles = new HashMap<>();
 					sstFiles.putAll(newSstFiles);
 					sstFiles.putAll(oldSstFiles);
 					stateBackend.materializedSstFiles.put(restoreStateHandle.getCheckpointId(), sstFiles);
@@ -1485,6 +1429,29 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			}
 		}
 
+		private void readAllStateData(
+			Map<StateHandleID, StreamStateHandle> stateHandleMap,
+			Path restoreInstancePath) throws IOException {
+
+			for (Map.Entry<StateHandleID, StreamStateHandle> entry : stateHandleMap.entrySet()) {
+				StateHandleID stateHandleID = entry.getKey();
+				StreamStateHandle remoteFileHandle = entry.getValue();
+				readStateData(new Path(restoreInstancePath, stateHandleID.toString()), remoteFileHandle);
+			}
+		}
+
+		private void createFileHardLinksInRestorePath(
+			Map<StateHandleID, StreamStateHandle> stateHandleMap,
+			Path restoreInstancePath) throws IOException {
+
+			for (StateHandleID stateHandleID : stateHandleMap.keySet()) {
+				String newSstFileName = stateHandleID.toString();
+				File restoreFile = new File(restoreInstancePath.getPath(), newSstFileName);
+				File targetFile = new File(stateBackend.instanceRocksDBPath, newSstFileName);
+				Files.createLink(targetFile.toPath(), restoreFile.toPath());
+			}
+		}
+
 		void restore(Collection<KeyedStateHandle> restoreStateHandles) throws Exception {
 
 			boolean hasExtraKeys = (restoreStateHandles.size() > 1 ||
@@ -1496,13 +1463,13 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 			for (KeyedStateHandle rawStateHandle : restoreStateHandles) {
 
-				if (! (rawStateHandle instanceof RocksDBIncrementalKeyedStateHandle)) {
+				if (! (rawStateHandle instanceof IncrementalKeyedStateHandle)) {
 					throw new IllegalStateException("Unexpected state handle type, " +
-						"expected " + RocksDBIncrementalKeyedStateHandle.class +
+						"expected " + IncrementalKeyedStateHandle.class +
 						", but found " + rawStateHandle.getClass());
 				}
 
-				RocksDBIncrementalKeyedStateHandle keyedStateHandle = (RocksDBIncrementalKeyedStateHandle) rawStateHandle;
+				IncrementalKeyedStateHandle keyedStateHandle = (IncrementalKeyedStateHandle) rawStateHandle;
 
 				restoreInstance(keyedStateHandle, hasExtraKeys);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java b/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
new file mode 100644
index 0000000..7245e61
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/StringBasedID.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.util;
+
+import java.io.Serializable;
+
+/**
+ * Base class for typed IDs that are internally represented by a string. This class is not intended
+ * for direct use, but should be subclassed for type-safety.
+ */
+public class StringBasedID implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Uses a String as internal representation
+	 */
+	private final String keyString;
+
+	/**
+	 * Protected constructor to enfore that subclassing.
+	 */
+	protected StringBasedID(String keyString) {
+		this.keyString = Preconditions.checkNotNull(keyString);
+	}
+
+	public String getKeyString() {
+		return keyString;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		StringBasedID that = (StringBasedID) o;
+		return keyString.equals(that.keyString);
+	}
+
+	@Override
+	public int hashCode() {
+		return keyString.hashCode();
+	}
+
+	@Override
+	public String toString() {
+		return keyString;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
index 1b5f2c6..b71418b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
@@ -20,14 +20,17 @@ package org.apache.flink.runtime.checkpoint.savepoint;
 
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.checkpoint.MasterState;
-import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.checkpoint.OperatorState;
 import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
+import org.apache.flink.runtime.state.StateHandleID;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
@@ -71,6 +74,8 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 	private static final byte FILE_STREAM_STATE_HANDLE = 2;
 	private static final byte KEY_GROUPS_HANDLE = 3;
 	private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4;
+	private static final byte INCREMENTAL_KEY_GROUPS_HANDLE = 5;
+	private static final byte PLACEHOLDER_STREAM_STATE_HANDLE = 6;
 
 	/** The singleton instance of the serializer */
 	public static final SavepointV2Serializer INSTANCE = new SavepointV2Serializer();
@@ -287,7 +292,6 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 
 		KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis);
 
-
 		return new OperatorSubtaskState(
 				nonPartitionableState,
 				operatorStateBackend,
@@ -311,19 +315,63 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 				dos.writeLong(keyGroupsStateHandle.getOffsetForKeyGroup(keyGroup));
 			}
 			serializeStreamStateHandle(keyGroupsStateHandle.getDelegateStateHandle(), dos);
+		} else if (stateHandle instanceof IncrementalKeyedStateHandle) {
+			IncrementalKeyedStateHandle incrementalKeyedStateHandle =
+				(IncrementalKeyedStateHandle) stateHandle;
+
+			dos.writeByte(INCREMENTAL_KEY_GROUPS_HANDLE);
+
+			dos.writeLong(incrementalKeyedStateHandle.getCheckpointId());
+			dos.writeUTF(incrementalKeyedStateHandle.getOperatorIdentifier());
+			dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getStartKeyGroup());
+			dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
+
+			serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos);
+
+			serializeStreamStateHandleMap(incrementalKeyedStateHandle.getCreatedSharedState(), dos);
+			serializeStreamStateHandleMap(incrementalKeyedStateHandle.getReferencedSharedState(), dos);
+			serializeStreamStateHandleMap(incrementalKeyedStateHandle.getPrivateState(), dos);
 		} else {
 			throw new IllegalStateException("Unknown KeyedStateHandle type: " + stateHandle.getClass());
 		}
 	}
 
+	private static void serializeStreamStateHandleMap(
+		Map<StateHandleID, StreamStateHandle> map,
+		DataOutputStream dos) throws IOException {
+		dos.writeInt(map.size());
+		for (Map.Entry<StateHandleID, StreamStateHandle> entry : map.entrySet()) {
+			dos.writeUTF(entry.getKey().toString());
+			serializeStreamStateHandle(entry.getValue(), dos);
+		}
+	}
+
+	private static Map<StateHandleID, StreamStateHandle> deserializeStreamStateHandleMap(
+		DataInputStream dis) throws IOException {
+
+		final int size = dis.readInt();
+		Map<StateHandleID, StreamStateHandle> result = new HashMap<>(size);
+
+		for (int i = 0; i < size; ++i) {
+			StateHandleID stateHandleID = new StateHandleID(dis.readUTF());
+			StreamStateHandle stateHandle = deserializeStreamStateHandle(dis);
+			result.put(stateHandleID, stateHandle);
+		}
+
+		return result;
+	}
+
 	private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException {
 		final int type = dis.readByte();
 		if (NULL_HANDLE == type) {
+
 			return null;
 		} else if (KEY_GROUPS_HANDLE == type) {
+
 			int startKeyGroup = dis.readInt();
 			int numKeyGroups = dis.readInt();
-			KeyGroupRange keyGroupRange = KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1);
+			KeyGroupRange keyGroupRange =
+				KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1);
 			long[] offsets = new long[numKeyGroups];
 			for (int i = 0; i < numKeyGroups; ++i) {
 				offsets[i] = dis.readLong();
@@ -332,6 +380,28 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 				keyGroupRange, offsets);
 			StreamStateHandle stateHandle = deserializeStreamStateHandle(dis);
 			return new KeyGroupsStateHandle(keyGroupRangeOffsets, stateHandle);
+		} else if (INCREMENTAL_KEY_GROUPS_HANDLE == type) {
+
+			long checkpointId = dis.readLong();
+			String operatorId = dis.readUTF();
+			int startKeyGroup = dis.readInt();
+			int numKeyGroups = dis.readInt();
+			KeyGroupRange keyGroupRange =
+				KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1);
+
+			StreamStateHandle metaDataStateHandle = deserializeStreamStateHandle(dis);
+			Map<StateHandleID, StreamStateHandle> createdStates = deserializeStreamStateHandleMap(dis);
+			Map<StateHandleID, StreamStateHandle> referencedStates = deserializeStreamStateHandleMap(dis);
+			Map<StateHandleID, StreamStateHandle> privateStates = deserializeStreamStateHandleMap(dis);
+
+			return new IncrementalKeyedStateHandle(
+				operatorId,
+				keyGroupRange,
+				checkpointId,
+				createdStates,
+				referencedStates,
+				privateStates,
+				metaDataStateHandle);
 		} else {
 			throw new IllegalStateException("Reading invalid KeyedStateHandle, type: " + type);
 		}
@@ -415,7 +485,10 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 			byte[] internalData = byteStreamStateHandle.getData();
 			dos.writeInt(internalData.length);
 			dos.write(byteStreamStateHandle.getData());
-
+		} else if (stateHandle instanceof PlaceholderStreamStateHandle) {
+			PlaceholderStreamStateHandle placeholder = (PlaceholderStreamStateHandle) stateHandle;
+			dos.writeByte(PLACEHOLDER_STREAM_STATE_HANDLE);
+			dos.writeLong(placeholder.getStateSize());
 		} else {
 			throw new IOException("Unknown implementation of StreamStateHandle: " + stateHandle.getClass());
 		}
@@ -437,6 +510,8 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 			byte[] data = new byte[numBytes];
 			dis.readFully(data);
 			return new ByteStreamStateHandle(handleName, data);
+		} else if (PLACEHOLDER_STREAM_STATE_HANDLE == type) {
+			return new PlaceholderStreamStateHandle(dis.readLong());
 		} else {
 			throw new IOException("Unknown implementation of StreamStateHandle, code: " + type);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java
new file mode 100644
index 0000000..706e219
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * The handle to states of an incremental snapshot.
+ * <p>
+ * The states contained in an incremental snapshot include
+ * <ul>
+ * <li> Created shared state which includes (the supposed to be) shared files produced since the last
+ * completed checkpoint. These files can be referenced by succeeding checkpoints if the
+ * checkpoint succeeds to complete. </li>
+ * <li> Referenced shared state which includes the shared files materialized in previous
+ * checkpoints. </li>
+ * <li> Private state which includes all other files, typically mutable, that cannot be shared by
+ * other checkpoints. </li>
+ * <li> Backend meta state which includes the information of existing states. </li>
+ * </ul>
+ *
+ * IMPORTANT: This class currently overrides equals and hash code only for testing purposes. They
+ * should not be called from production code. This means this class is also not suited to serve as
+ * a key, e.g. in hash maps.
+ */
+public class IncrementalKeyedStateHandle implements KeyedStateHandle {
+
+	private static final Logger LOG = LoggerFactory.getLogger(IncrementalKeyedStateHandle.class);
+
+	private static final long serialVersionUID = -8328808513197388231L;
+
+	/**
+	 * The operator instance identifier for this handle
+	 */
+	private final String operatorIdentifier;
+
+	/**
+	 * The key-group range covered by this state handle
+	 */
+	private final KeyGroupRange keyGroupRange;
+
+	/**
+	 * The checkpoint Id
+	 */
+	private final long checkpointId;
+
+	/**
+	 * State that the incremental checkpoint created new
+	 */
+	private final Map<StateHandleID, StreamStateHandle> createdSharedState;
+
+	/**
+	 * State that the incremental checkpoint references from previous checkpoints
+	 */
+	private final Map<StateHandleID, StreamStateHandle> referencedSharedState;
+
+	/**
+	 * Private state in the incremental checkpoint
+	 */
+	private final Map<StateHandleID, StreamStateHandle> privateState;
+
+	/**
+	 * Primary meta data state of the incremental checkpoint
+	 */
+	private final StreamStateHandle metaStateHandle;
+
+	/**
+	 * True if the state handle has already registered shared states.
+	 * <p>
+	 * Once the shared states are registered, it's the {@link SharedStateRegistry}'s
+	 * responsibility to maintain the shared states. But in the cases where the
+	 * state handle is discarded before performing the registration, the handle
+	 * should delete all the shared states created by it.
+	 */
+	private boolean registered;
+
+	public IncrementalKeyedStateHandle(
+		String operatorIdentifier,
+		KeyGroupRange keyGroupRange,
+		long checkpointId,
+		Map<StateHandleID, StreamStateHandle> createdSharedState,
+		Map<StateHandleID, StreamStateHandle> referencedSharedState,
+		Map<StateHandleID, StreamStateHandle> privateState,
+		StreamStateHandle metaStateHandle) {
+
+		this.operatorIdentifier = Preconditions.checkNotNull(operatorIdentifier);
+		this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
+		this.checkpointId = checkpointId;
+		this.createdSharedState = Preconditions.checkNotNull(createdSharedState);
+		this.referencedSharedState = Preconditions.checkNotNull(referencedSharedState);
+		this.privateState = Preconditions.checkNotNull(privateState);
+		this.metaStateHandle = Preconditions.checkNotNull(metaStateHandle);
+		this.registered = false;
+	}
+
+	@Override
+	public KeyGroupRange getKeyGroupRange() {
+		return keyGroupRange;
+	}
+
+	public long getCheckpointId() {
+		return checkpointId;
+	}
+
+	public Map<StateHandleID, StreamStateHandle> getCreatedSharedState() {
+		return createdSharedState;
+	}
+
+	public Map<StateHandleID, StreamStateHandle> getReferencedSharedState() {
+		return referencedSharedState;
+	}
+
+	public Map<StateHandleID, StreamStateHandle> getPrivateState() {
+		return privateState;
+	}
+
+	public StreamStateHandle getMetaStateHandle() {
+		return metaStateHandle;
+	}
+
+	public String getOperatorIdentifier() {
+		return operatorIdentifier;
+	}
+
+	@Override
+	public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+		if (this.keyGroupRange.getIntersection(keyGroupRange) != KeyGroupRange.EMPTY_KEY_GROUP_RANGE) {
+			return this;
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public void discardState() throws Exception {
+
+		Preconditions.checkState(!registered, "Attempt to dispose a registered composite state with registered shared state. Must unregister first.");
+
+		try {
+			metaStateHandle.discardState();
+		} catch (Exception e) {
+			LOG.warn("Could not properly discard meta data.", e);
+		}
+
+		try {
+			StateUtil.bestEffortDiscardAllStateObjects(privateState.values());
+		} catch (Exception e) {
+			LOG.warn("Could not properly discard misc file states.", e);
+		}
+
+		try {
+			StateUtil.bestEffortDiscardAllStateObjects(createdSharedState.values());
+		} catch (Exception e) {
+			LOG.warn("Could not properly discard new sst file states.", e);
+		}
+
+	}
+
+	@Override
+	public long getStateSize() {
+		long size = getPrivateStateSize();
+
+		for (StreamStateHandle oldSstFileHandle : referencedSharedState.values()) {
+			size += oldSstFileHandle.getStateSize();
+		}
+
+		return size;
+	}
+
+	/**
+	 * Returns the size of the state that is privately owned by this handle.
+	 */
+	public long getPrivateStateSize() {
+		long size = StateUtil.getStateSize(metaStateHandle);
+
+		for (StreamStateHandle newSstFileHandle : createdSharedState.values()) {
+			size += newSstFileHandle.getStateSize();
+		}
+
+		for (StreamStateHandle miscFileHandle : privateState.values()) {
+			size += miscFileHandle.getStateSize();
+		}
+
+		return size;
+	}
+
+	@Override
+	public void registerSharedStates(SharedStateRegistry stateRegistry) {
+
+		Preconditions.checkState(!registered, "The state handle has already registered its shared states.");
+
+		for (Map.Entry<StateHandleID, StreamStateHandle> newSstFileEntry : createdSharedState.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(newSstFileEntry.getKey());
+
+			SharedStateRegistry.Result result =
+				stateRegistry.registerNewReference(registryKey, newSstFileEntry.getValue());
+
+			// We update our reference with the result from the registry, to prevent the following
+			// problem:
+			// A previous checkpoint n has already registered the state. This can happen if a
+			// following checkpoint (n + x) wants to reference the same state before the backend got
+			// notified that checkpoint n completed. In this case, the shared registry did
+			// deduplication and returns the previous reference.
+			newSstFileEntry.setValue(result.getReference());
+		}
+
+		for (Map.Entry<StateHandleID, StreamStateHandle> oldSstFileName : referencedSharedState.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(oldSstFileName.getKey());
+
+			SharedStateRegistry.Result result = stateRegistry.obtainReference(registryKey);
+
+			// Again we update our state handle with the result from the registry, thus replacing
+			// placeholder state handles with the originals.
+			oldSstFileName.setValue(result.getReference());
+		}
+
+		// Migrate state from unregistered to registered, so that it will not count as private state
+		// for #discardState() from now.
+		referencedSharedState.putAll(createdSharedState);
+		createdSharedState.clear();
+
+		registered = true;
+	}
+
+	@Override
+	public void unregisterSharedStates(SharedStateRegistry stateRegistry) {
+
+		Preconditions.checkState(registered, "The state handle has not registered its shared states yet.");
+
+		for (Map.Entry<StateHandleID, StreamStateHandle> newSstFileEntry : createdSharedState.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(newSstFileEntry.getKey());
+			stateRegistry.releaseReference(registryKey);
+		}
+
+		for (Map.Entry<StateHandleID, StreamStateHandle> oldSstFileEntry : referencedSharedState.entrySet()) {
+			SharedStateRegistryKey registryKey =
+				createSharedStateRegistryKeyFromFileName(oldSstFileEntry.getKey());
+			stateRegistry.releaseReference(registryKey);
+		}
+
+		registered = false;
+	}
+
+	private SharedStateRegistryKey createSharedStateRegistryKeyFromFileName(StateHandleID shId) {
+		return new SharedStateRegistryKey(operatorIdentifier + '-' + keyGroupRange, shId);
+	}
+
+	/**
+	 * This method is should only be called in tests! This should never serve as key in a hash map.
+	 */
+	@VisibleForTesting
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		IncrementalKeyedStateHandle that = (IncrementalKeyedStateHandle) o;
+
+		if (getCheckpointId() != that.getCheckpointId()) {
+			return false;
+		}
+		if (!getOperatorIdentifier().equals(that.getOperatorIdentifier())) {
+			return false;
+		}
+		if (!getKeyGroupRange().equals(that.getKeyGroupRange())) {
+			return false;
+		}
+		if (!getCreatedSharedState().equals(that.getCreatedSharedState())) {
+			return false;
+		}
+		if (!getReferencedSharedState().equals(that.getReferencedSharedState())) {
+			return false;
+		}
+		if (!getPrivateState().equals(that.getPrivateState())) {
+			return false;
+		}
+		return getMetaStateHandle().equals(that.getMetaStateHandle());
+	}
+
+	/**
+	 * This method should only be called in tests! This should never serve as key in a hash map.
+	 */
+	@VisibleForTesting
+	@Override
+	public int hashCode() {
+		int result = getOperatorIdentifier().hashCode();
+		result = 31 * result + getKeyGroupRange().hashCode();
+		result = 31 * result + (int) (getCheckpointId() ^ (getCheckpointId() >>> 32));
+		result = 31 * result + getCreatedSharedState().hashCode();
+		result = 31 * result + getReferencedSharedState().hashCode();
+		result = 31 * result + getPrivateState().hashCode();
+		result = 31 * result + getMetaStateHandle().hashCode();
+		return result;
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java
new file mode 100644
index 0000000..2136061
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+/**
+ * A placeholder state handle for shared state that will replaced by an original that was
+ * created in a previous checkpoint. So we don't have to send the handle twice, e.g. in
+ * case of {@link ByteStreamStateHandle}. To be used in the referenced states of
+ * {@link IncrementalKeyedStateHandle}.
+ * <p>
+ * IMPORTANT: This class currently overrides equals and hash code only for testing purposes. They
+ * should not be called from production code. This means this class is also not suited to serve as
+ * a key, e.g. in hash maps.
+ */
+public class PlaceholderStreamStateHandle implements StreamStateHandle {
+
+	private static final long serialVersionUID = 1L;
+
+	/** We remember the size of the original file for which this is a placeholder */
+	private final long originalSize;
+
+	public PlaceholderStreamStateHandle(long originalSize) {
+		this.originalSize = originalSize;
+	}
+
+	@Override
+	public FSDataInputStream openInputStream() {
+		throw new UnsupportedOperationException(
+			"This is only a placeholder to be replaced by a real StreamStateHandle in the checkpoint coordinator.");
+	}
+
+	@Override
+	public void discardState() throws Exception {
+		// nothing to do.
+	}
+
+	@Override
+	public long getStateSize() {
+		return originalSize;
+	}
+
+	/**
+	 * This method is should only be called in tests! This should never serve as key in a hash map.
+	 */
+	@VisibleForTesting
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		PlaceholderStreamStateHandle that = (PlaceholderStreamStateHandle) o;
+
+		return originalSize == that.originalSize;
+	}
+
+	/**
+	 * This method is should only be called in tests! This should never serve as key in a hash map.
+	 */
+	@VisibleForTesting
+	@Override
+	public int hashCode() {
+		return (int) (originalSize ^ (originalSize >>> 32));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateHandle.java
deleted file mode 100644
index c8c4046..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateHandle.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.state;
-
-/**
- * A handle to those states that are referenced by different checkpoints.
- *
- * <p> Each shared state handle is identified by a unique key. Two shared states
- * are considered equal if their keys are identical.
- *
- * <p> All shared states are registered at the {@link SharedStateRegistry} once
- * they are received by the {@link org.apache.flink.runtime.checkpoint.CheckpointCoordinator}
- * and will be unregistered when the checkpoints are discarded. A shared state
- * will be discarded once it is not referenced by any checkpoint. A shared state
- * should not be referenced any more if it has been discarded.
- */
-public interface SharedStateHandle extends StateObject {
-
-	/**
-	 * Return the identifier of the shared state.
-	 */
-	String getRegistrationKey();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
index dbf4642..9cfdec7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
@@ -31,7 +31,7 @@ import java.util.concurrent.Executor;
 /**
  * A {@code SharedStateRegistry} will be deployed in the 
  * {@link org.apache.flink.runtime.checkpoint.CompletedCheckpointStore} to
- * maintain the reference count of {@link SharedStateHandle}s which are shared
+ * maintain the reference count of {@link StreamStateHandle}s which are shared
  * among different incremental checkpoints.
  */
 public class SharedStateRegistry {

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
index 9e59359..58262ca 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryKey.java
@@ -18,9 +18,8 @@
 
 package org.apache.flink.runtime.state;
 
-import org.apache.flink.util.Preconditions;
-
-import java.io.Serializable;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.StringBasedID;
 
 /**
  * This class represents a key that uniquely identifies (on a logical level) state handles for
@@ -28,41 +27,16 @@ import java.io.Serializable;
  * be the same should have the same {@link SharedStateRegistryKey}. The meaning of logical
  * equivalence is up to the application.
  */
-public class SharedStateRegistryKey implements Serializable {
+public class SharedStateRegistryKey extends StringBasedID {
 
 	private static final long serialVersionUID = 1L;
 
-	/** Uses a String as internal representation */
-	private final String keyString;
-
-	public SharedStateRegistryKey(String keyString) {
-		this.keyString = Preconditions.checkNotNull(keyString);
-	}
-
-	public String getKeyString() {
-		return keyString;
+	public SharedStateRegistryKey(String prefix, StateHandleID stateHandleID) {
+		super(prefix + '-' + stateHandleID);
 	}
 
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		SharedStateRegistryKey that = (SharedStateRegistryKey) o;
-		return keyString.equals(that.keyString);
-	}
-
-	@Override
-	public int hashCode() {
-		return keyString.hashCode();
-	}
-
-	@Override
-	public String toString() {
-		return keyString;
+	@VisibleForTesting
+	public SharedStateRegistryKey(String keyString) {
+		super(keyString);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandle.java
deleted file mode 100644
index b736252..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandle.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.state;
-
-/**
- * StateHandle is a general handle interface meant to abstract operator state fetching. 
- * A StateHandle implementation can for example include the state itself in cases where the state 
- * is lightweight or fetching it lazily from some external storage when the state is too large.
- */
-public interface StateHandle<T> extends StateObject {
-
-	/**
-	 * This retrieves and return the state represented by the handle.
-	 *
-	 * @param userCodeClassLoader Class loader for deserializing user code specific classes
-	 *
-	 * @return The state represented by the handle.
-	 * @throws java.lang.Exception Thrown, if the state cannot be fetched.
-	 */
-	T getState(ClassLoader userCodeClassLoader) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleID.java
new file mode 100644
index 0000000..5e95cff
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleID.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.util.StringBasedID;
+
+/**
+ * Unique ID that allows for logical comparison between state handles.
+ * <p>
+ * Two state handles that are considered as logically equal should always return the same ID
+ * (whatever logically equal means is up to the implementation). For example, this could be based
+ * on the string representation of the full filepath for a state that is based on a file.
+ */
+public class StateHandleID extends StringBasedID {
+
+	private static final long serialVersionUID = 1L;
+
+	public StateHandleID(String keyString) {
+		super(keyString);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
index ba77dbc..b63782d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
@@ -27,10 +27,15 @@ import org.apache.flink.runtime.checkpoint.TaskState;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.state.ChainedStateHandle;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle.StateMetaInfo;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
+import org.apache.flink.runtime.state.StateHandleID;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare;
 import org.apache.flink.util.StringUtils;
@@ -41,6 +46,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.UUID;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -78,6 +84,7 @@ public class CheckpointTestUtils {
 
 			boolean hasKeyedBackend = random.nextInt(4) != 0;
 			boolean hasKeyedStream = random.nextInt(4) != 0;
+			boolean isIncremental = random.nextInt(3) == 0;
 
 			for (int subtaskIdx = 0; subtaskIdx < numSubtasksPerTask; subtaskIdx++) {
 
@@ -108,21 +115,19 @@ public class CheckpointTestUtils {
 					operatorStateHandleStream = new OperatorStateHandle(offsetsMap, operatorStateStream);
 				}
 
-				KeyGroupsStateHandle keyedStateBackend = null;
-				KeyGroupsStateHandle keyedStateStream = null;
+				KeyedStateHandle keyedStateBackend = null;
+				KeyedStateHandle keyedStateStream = null;
 
 				if (hasKeyedBackend) {
-					keyedStateBackend = new KeyGroupsStateHandle(
-							new KeyGroupRangeOffsets(1, 1, new long[]{42}),
-							new TestByteStreamStateHandleDeepCompare("c", "Hello"
-									.getBytes(ConfigConstants.DEFAULT_CHARSET)));
+					if (isIncremental) {
+						keyedStateBackend = createDummyIncrementalKeyedStateHandle(random);
+					} else {
+						keyedStateBackend = createDummyKeyGroupStateHandle(random);
+					}
 				}
 
 				if (hasKeyedStream) {
-					keyedStateStream = new KeyGroupsStateHandle(
-							new KeyGroupRangeOffsets(1, 1, new long[]{23}),
-							new TestByteStreamStateHandleDeepCompare("d", "World"
-									.getBytes(ConfigConstants.DEFAULT_CHARSET)));
+					keyedStateStream = createDummyKeyGroupStateHandle(random);
 				}
 
 				taskState.putState(subtaskIdx, new OperatorSubtaskState(
@@ -210,17 +215,11 @@ public class CheckpointTestUtils {
 				KeyGroupsStateHandle keyedStateStream = null;
 
 				if (hasKeyedBackend) {
-					keyedStateBackend = new KeyGroupsStateHandle(
-							new KeyGroupRangeOffsets(1, 1, new long[]{42}),
-							new TestByteStreamStateHandleDeepCompare("c", "Hello"
-								.getBytes(ConfigConstants.DEFAULT_CHARSET)));
+					keyedStateBackend = createDummyKeyGroupStateHandle(random);
 				}
 
 				if (hasKeyedStream) {
-					keyedStateStream = new KeyGroupsStateHandle(
-							new KeyGroupRangeOffsets(1, 1, new long[]{23}),
-							new TestByteStreamStateHandleDeepCompare("d", "World"
-								.getBytes(ConfigConstants.DEFAULT_CHARSET)));
+					keyedStateStream = createDummyKeyGroupStateHandle(random);
 				}
 
 				taskState.putState(subtaskIdx, new SubtaskState(
@@ -272,4 +271,56 @@ public class CheckpointTestUtils {
 
 	/** utility class, not meant to be instantiated */
 	private CheckpointTestUtils() {}
+
+
+	private static IncrementalKeyedStateHandle createDummyIncrementalKeyedStateHandle(Random rnd) {
+		return new IncrementalKeyedStateHandle(
+			createRandomUUID(rnd).toString(),
+			new KeyGroupRange(1, 1),
+			42L,
+			createRandomOwnedHandleMap(rnd),
+			createRandomReferencedHandleMap(rnd),
+			createRandomOwnedHandleMap(rnd),
+			createDummyStreamStateHandle(rnd));
+	}
+
+	private static Map<StateHandleID, StreamStateHandle> createRandomOwnedHandleMap(Random rnd) {
+		final int size = rnd.nextInt(4);
+		Map<StateHandleID, StreamStateHandle> result = new HashMap<>(size);
+		for (int i = 0; i < size; ++i) {
+			StateHandleID randomId = new StateHandleID(createRandomUUID(rnd).toString());
+			StreamStateHandle stateHandle = createDummyStreamStateHandle(rnd);
+			result.put(randomId, stateHandle);
+		}
+
+		return result;
+	}
+
+	private static Map<StateHandleID, StreamStateHandle> createRandomReferencedHandleMap(Random rnd) {
+		final int size = rnd.nextInt(4);
+		Map<StateHandleID, StreamStateHandle> result = new HashMap<>(size);
+		for (int i = 0; i < size; ++i) {
+			StateHandleID randomId = new StateHandleID(createRandomUUID(rnd).toString());
+			result.put(randomId, new PlaceholderStreamStateHandle(rnd.nextInt(1024)));
+		}
+
+		return result;
+	}
+
+	private static KeyGroupsStateHandle createDummyKeyGroupStateHandle(Random rnd) {
+		return new KeyGroupsStateHandle(
+			new KeyGroupRangeOffsets(1, 1, new long[]{rnd.nextInt(1024)}),
+			createDummyStreamStateHandle(rnd));
+	}
+
+	private static StreamStateHandle createDummyStreamStateHandle(Random rnd) {
+		return new TestByteStreamStateHandleDeepCompare(
+			String.valueOf(createRandomUUID(rnd)),
+			String.valueOf(createRandomUUID(rnd)).getBytes(ConfigConstants.DEFAULT_CHARSET));
+	}
+
+	private static UUID createRandomUUID(Random rnd) {
+		return new UUID(rnd.nextLong(), rnd.nextLong());
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2SerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2SerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2SerializerTest.java
index 154d761..602390b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2SerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2SerializerTest.java
@@ -24,7 +24,6 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.checkpoint.MasterState;
 import org.apache.flink.runtime.checkpoint.OperatorState;
-
 import org.junit.Test;
 
 import java.io.DataInputStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/7c6f3485/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateUtilTest.java
deleted file mode 100644
index d6966d0..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateUtilTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.state;
-
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import java.util.concurrent.RunnableFuture;
-
-public class StateUtilTest extends TestLogger {
-
-	/**
-	 * Tests that {@link StateUtil#discardStateFuture} can handle state futures with null value.
-	 */
-	@Test
-	public void testDiscardRunnableFutureWithNullValue() throws Exception {
-		RunnableFuture<StateHandle<?>> stateFuture = DoneFuture.nullValue();
-		StateUtil.discardStateFuture(stateFuture);
-	}
-}


[08/14] flink git commit: [FLINK-6414] [build] Use scala.binary.version in place of change-scala-version.sh

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-gelly-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/pom.xml b/flink-libraries/flink-gelly-examples/pom.xml
index 009781f..68ad050 100644
--- a/flink-libraries/flink-gelly-examples/pom.xml
+++ b/flink-libraries/flink-gelly-examples/pom.xml
@@ -27,7 +27,7 @@
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-gelly-examples_2.10</artifactId>
+	<artifactId>flink-gelly-examples_${scala.binary.version}</artifactId>
 	<name>flink-gelly-examples</name>
 	<packaging>jar</packaging>
 
@@ -42,22 +42,22 @@
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-gelly_2.10</artifactId>
+			<artifactId>flink-gelly_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-gelly-scala_2.10</artifactId>
+			<artifactId>flink-gelly-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -94,14 +94,14 @@
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-gelly-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/pom.xml b/flink-libraries/flink-gelly-scala/pom.xml
index 256fc53..35b2188 100644
--- a/flink-libraries/flink-gelly-scala/pom.xml
+++ b/flink-libraries/flink-gelly-scala/pom.xml
@@ -28,7 +28,7 @@ under the License.
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>flink-gelly-scala_2.10</artifactId>
+    <artifactId>flink-gelly-scala_${scala.binary.version}</artifactId>
     <name>flink-gelly-scala</name>
 
     <packaging>jar</packaging>
@@ -38,21 +38,21 @@ under the License.
         <!-- core dependencies -->
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-scala_2.10</artifactId>
+            <artifactId>flink-scala_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-clients_2.10</artifactId>
+            <artifactId>flink-clients_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-gelly_2.10</artifactId>
+            <artifactId>flink-gelly_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
         </dependency>
 
@@ -87,7 +87,7 @@ under the License.
         
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-tests_2.10</artifactId>
+            <artifactId>flink-tests_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
             <type>test-jar</type>
@@ -95,7 +95,7 @@ under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils_2.10</artifactId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-gelly/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml
index d656bb3..d620a66 100644
--- a/flink-libraries/flink-gelly/pom.xml
+++ b/flink-libraries/flink-gelly/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-gelly_2.10</artifactId>
+	<artifactId>flink-gelly_${scala.binary.version}</artifactId>
 	<name>flink-gelly</name>
 
 	<packaging>jar</packaging>
@@ -47,7 +47,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -62,7 +62,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
@@ -77,7 +77,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-optimizer_2.10</artifactId>
+			<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-ml/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/pom.xml b/flink-libraries/flink-ml/pom.xml
index 8aa84d2..0f203b9 100644
--- a/flink-libraries/flink-ml/pom.xml
+++ b/flink-libraries/flink-ml/pom.xml
@@ -28,7 +28,7 @@
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-ml_2.10</artifactId>
+	<artifactId>flink-ml_${scala.binary.version}</artifactId>
 	<name>flink-ml</name>
 
 	<packaging>jar</packaging>
@@ -39,7 +39,7 @@
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -86,14 +86,14 @@
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-python/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-python/pom.xml b/flink-libraries/flink-python/pom.xml
index 9d2f0c3..5b6405e 100644
--- a/flink-libraries/flink-python/pom.xml
+++ b/flink-libraries/flink-python/pom.xml
@@ -27,7 +27,7 @@ under the License.
         <relativePath>..</relativePath>
     </parent>
 
-    <artifactId>flink-python_2.10</artifactId>
+    <artifactId>flink-python_${scala.binary.version}</artifactId>
     <name>flink-python</name>
     <packaging>jar</packaging>
 
@@ -69,7 +69,7 @@ under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-runtime_2.10</artifactId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
 			<scope>provided</scope>
         </dependency>
@@ -78,7 +78,7 @@ under the License.
         
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils_2.10</artifactId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-libraries/flink-table/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml
index 8fa2ed2..a34fa1b 100644
--- a/flink-libraries/flink-table/pom.xml
+++ b/flink-libraries/flink-table/pom.xml
@@ -27,7 +27,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-table_2.10</artifactId>
+	<artifactId>flink-table_${scala.binary.version}</artifactId>
 	<name>flink-table</name>
 
 	<packaging>jar</packaging>
@@ -38,7 +38,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -114,14 +114,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -137,20 +137,20 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-statebackend-rocksdb_2.10</artifactId>
+			<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-mesos/pom.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml
index d03144f..a1ef900 100644
--- a/flink-mesos/pom.xml
+++ b/flink-mesos/pom.xml
@@ -27,7 +27,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 	
-	<artifactId>flink-mesos_2.10</artifactId>
+	<artifactId>flink-mesos_${scala.binary.version}</artifactId>
 	<name>flink-mesos</name>
 	<packaging>jar</packaging>
 
@@ -38,13 +38,13 @@ under the License.
     <dependencies>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -130,7 +130,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -138,7 +138,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-metrics/flink-metrics-datadog/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-datadog/pom.xml b/flink-metrics/flink-metrics-datadog/pom.xml
index e2bd6fa..1b1d9ad 100644
--- a/flink-metrics/flink-metrics-datadog/pom.xml
+++ b/flink-metrics/flink-metrics-datadog/pom.xml
@@ -63,7 +63,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-metrics/flink-metrics-dropwizard/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml
index 2ba12e6..631d68e 100644
--- a/flink-metrics/flink-metrics-dropwizard/pom.xml
+++ b/flink-metrics/flink-metrics-dropwizard/pom.xml
@@ -58,7 +58,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-metrics/flink-metrics-jmx/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-jmx/pom.xml b/flink-metrics/flink-metrics-jmx/pom.xml
index d738fc4..1e8ecc8 100644
--- a/flink-metrics/flink-metrics-jmx/pom.xml
+++ b/flink-metrics/flink-metrics-jmx/pom.xml
@@ -49,7 +49,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
@@ -65,7 +65,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-metrics/flink-metrics-statsd/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml
index 80cfa32..5b77f8d 100644
--- a/flink-metrics/flink-metrics-statsd/pom.xml
+++ b/flink-metrics/flink-metrics-statsd/pom.xml
@@ -51,7 +51,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-optimizer/pom.xml
----------------------------------------------------------------------
diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml
index c347082..499487b 100644
--- a/flink-optimizer/pom.xml
+++ b/flink-optimizer/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-optimizer_2.10</artifactId>
+	<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 	<name>flink-optimizer</name>
 
 	<packaging>jar</packaging>
@@ -46,7 +46,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index 2fa1835..9991d2c 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -82,12 +82,12 @@ under the License.
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${flink.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${flink.version}</version>
 		</dependency>
 
@@ -124,13 +124,13 @@ under the License.
 				</dependency>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-streaming-java_2.10</artifactId>
+					<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-clients_2.10</artifactId>
+					<artifactId>flink-clients_${scala.binary.version}</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>
@@ -201,20 +201,20 @@ under the License.
 									<exclude>org.apache.flink:flink-shaded-curator-recipes</exclude>
 									<exclude>org.apache.flink:flink-core</exclude>
 									<exclude>org.apache.flink:flink-java</exclude>
-									<exclude>org.apache.flink:flink-scala_2.10</exclude>
-									<exclude>org.apache.flink:flink-runtime_2.10</exclude>
-									<exclude>org.apache.flink:flink-optimizer_2.10</exclude>
-									<exclude>org.apache.flink:flink-clients_2.10</exclude>
-									<exclude>org.apache.flink:flink-avro_2.10</exclude>
-									<exclude>org.apache.flink:flink-examples-batch_2.10</exclude>
-									<exclude>org.apache.flink:flink-examples-streaming_2.10</exclude>
-									<exclude>org.apache.flink:flink-streaming-java_2.10</exclude>
-									<exclude>org.apache.flink:flink-streaming-scala_2.10</exclude>
-									<exclude>org.apache.flink:flink-scala-shell_2.10</exclude>
+									<exclude>org.apache.flink:flink-scala_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-runtime_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-optimizer_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-clients_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-avro_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-examples-batch_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-examples-streaming_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-streaming-java_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-streaming-scala_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-scala-shell_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-python</exclude>
 									<exclude>org.apache.flink:flink-metrics-core</exclude>
 									<exclude>org.apache.flink:flink-metrics-jmx</exclude>
-									<exclude>org.apache.flink:flink-statebackend-rocksdb_2.10</exclude>
+									<exclude>org.apache.flink:flink-statebackend-rocksdb_${scala.binary.version}</exclude>
 
 									<!-- Also exclude very big transitive dependencies of Flink
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index 189bbce..2139c6b 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -78,17 +78,17 @@ under the License.
 		<!-- Apache Flink dependencies -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${flink.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${flink.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${flink.version}</version>
 		</dependency>
 		
@@ -116,19 +116,19 @@ under the License.
 			<dependencies>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-scala_2.10</artifactId>
+					<artifactId>flink-scala_${scala.binary.version}</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-streaming-scala_2.10</artifactId>
+					<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-clients_2.10</artifactId>
+					<artifactId>flink-clients_${scala.binary.version}</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>
@@ -202,20 +202,20 @@ under the License.
 									<exclude>org.apache.flink:flink-shaded-curator-recipes</exclude>
 									<exclude>org.apache.flink:flink-core</exclude>
 									<exclude>org.apache.flink:flink-java</exclude>
-									<exclude>org.apache.flink:flink-scala_2.10</exclude>
-									<exclude>org.apache.flink:flink-runtime_2.10</exclude>
-									<exclude>org.apache.flink:flink-optimizer_2.10</exclude>
-									<exclude>org.apache.flink:flink-clients_2.10</exclude>
-									<exclude>org.apache.flink:flink-avro_2.10</exclude>
-									<exclude>org.apache.flink:flink-examples-batch_2.10</exclude>
-									<exclude>org.apache.flink:flink-examples-streaming_2.10</exclude>
-									<exclude>org.apache.flink:flink-streaming-java_2.10</exclude>
-									<exclude>org.apache.flink:flink-streaming-scala_2.10</exclude>
-									<exclude>org.apache.flink:flink-scala-shell_2.10</exclude>
+									<exclude>org.apache.flink:flink-scala_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-runtime_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-optimizer_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-clients_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-avro_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-examples-batch_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-examples-streaming_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-streaming-java_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-streaming-scala_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-scala-shell_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-python</exclude>
 									<exclude>org.apache.flink:flink-metrics-core</exclude>
 									<exclude>org.apache.flink:flink-metrics-jmx</exclude>
-									<exclude>org.apache.flink:flink-statebackend-rocksdb_2.10</exclude>
+									<exclude>org.apache.flink:flink-statebackend-rocksdb_${scala.binary.version}</exclude>
 
 									<!-- Also exclude very big transitive dependencies of Flink
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-runtime-web/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml
index e7870f3..8a4671b 100644
--- a/flink-runtime-web/pom.xml
+++ b/flink-runtime-web/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-runtime-web_2.10</artifactId>
+	<artifactId>flink-runtime-web_${scala.binary.version}</artifactId>
 	<name>flink-runtime-web</name>
 
 	<packaging>jar</packaging>
@@ -42,12 +42,12 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -99,7 +99,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 8abdc3c..a13a985 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-runtime_2.10</artifactId>
+	<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 	<name>flink-runtime</name>
 
 	<packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-scala-shell/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala-shell/pom.xml b/flink-scala-shell/pom.xml
index 0aadfc6..c9acbec 100644
--- a/flink-scala-shell/pom.xml
+++ b/flink-scala-shell/pom.xml
@@ -27,7 +27,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-scala-shell_2.10</artifactId>
+	<artifactId>flink-scala-shell_${scala.binary.version}</artifactId>
 	<name>flink-scala-shell</name>
 
 	<packaging>jar</packaging>
@@ -44,19 +44,19 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala_2.10</artifactId>
+			<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -82,7 +82,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
@@ -214,14 +214,14 @@ under the License.
 			<id>scala-2.10</id>
 			<activation>
 				<property>
-					<!-- this is the default scala profile -->
+					<!-- only required for Scala 2.10 -->
 					<name>!scala-2.11</name>
 				</property>
 			</activation>
 			<dependencies>
 				<dependency>
 					<groupId>org.scalamacros</groupId>
-					<artifactId>quasiquotes_${scala.binary.version}</artifactId>
+					<artifactId>quasiquotes_2.10</artifactId>
 					<version>${scala.macros.version}</version>
 				</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index 7be1e29..7fad487 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -28,7 +28,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-scala_2.10</artifactId>
+	<artifactId>flink-scala_${scala.binary.version}</artifactId>
 	<name>flink-scala</name>
 	<packaging>jar</packaging>
 
@@ -82,7 +82,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -241,14 +241,14 @@ under the License.
 			<id>scala-2.10</id>
 			<activation>
 				<property>
-					<!-- this is the default scala profile -->
+					<!-- only required for Scala 2.10 -->
 					<name>!scala-2.11</name>
 				</property>
 			</activation>
 			<dependencies>
 				<dependency>
 					<groupId>org.scalamacros</groupId>
-					<artifactId>quasiquotes_${scala.binary.version}</artifactId>
+					<artifactId>quasiquotes_2.10</artifactId>
 					<version>${scala.macros.version}</version>
 				</dependency>
 			</dependencies>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-streaming-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml
index ea987af..46142de 100644
--- a/flink-streaming-java/pom.xml
+++ b/flink-streaming-java/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-streaming-java_2.10</artifactId>
+	<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 	<name>flink-streaming-java</name>
 
 	<packaging>jar</packaging>
@@ -46,13 +46,13 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -85,7 +85,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-streaming-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml
index 4ec12bd..586dca7 100644
--- a/flink-streaming-scala/pom.xml
+++ b/flink-streaming-scala/pom.xml
@@ -28,7 +28,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-streaming-scala_2.10</artifactId>
+	<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
 	<name>flink-streaming-scala</name>
 	<packaging>jar</packaging>
 
@@ -38,13 +38,13 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -85,14 +85,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -100,7 +100,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests_2.10</artifactId>
+			<artifactId>flink-tests_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -108,7 +108,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-test-utils-parent/flink-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml
index 36bdceb..4f53794 100644
--- a/flink-test-utils-parent/flink-test-utils/pom.xml
+++ b/flink-test-utils-parent/flink-test-utils/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-test-utils_2.10</artifactId>
+	<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 	<name>flink-test-utils</name>
 
 	<packaging>jar</packaging>
@@ -45,21 +45,21 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>compile</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>compile</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>compile</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index b67edbb..3c0b184 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -30,7 +30,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-tests_2.10</artifactId>
+	<artifactId>flink-tests_${scala.binary.version}</artifactId>
 	<name>flink-tests</name>
 
 	<packaging>jar</packaging>
@@ -54,28 +54,28 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-optimizer_2.10</artifactId>
+			<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime-web_2.10</artifactId>
+			<artifactId>flink-runtime-web_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime-web_2.10</artifactId>
+			<artifactId>flink-runtime-web_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -83,7 +83,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
@@ -97,14 +97,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala_2.10</artifactId>
+			<artifactId>flink-scala_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -112,21 +112,21 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-examples-batch_2.10</artifactId>
+			<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
@@ -141,14 +141,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_2.10</artifactId>
+			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -156,7 +156,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_2.10</artifactId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -164,14 +164,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_2.10</artifactId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-optimizer_2.10</artifactId>
+			<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -179,7 +179,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -224,7 +224,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-statebackend-rocksdb_2.10</artifactId>
+			<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-yarn-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index f296e8d..626d886 100644
--- a/flink-yarn-tests/pom.xml
+++ b/flink-yarn-tests/pom.xml
@@ -33,7 +33,7 @@ under the License.
 	We need the YARN fat jar build by flink-dist for the tests.
 	-->
 	
-	<artifactId>flink-yarn-tests_2.10</artifactId>
+	<artifactId>flink-yarn-tests_${scala.binary.version}</artifactId>
 	<name>flink-yarn-tests</name>
 	<packaging>jar</packaging>
 
@@ -43,14 +43,14 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>
@@ -59,21 +59,21 @@ under the License.
 		<!-- Needed for the streaming wordcount example -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_2.10</artifactId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-yarn_2.10</artifactId>
+			<artifactId>flink-yarn_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-yarn_2.10</artifactId>
+			<artifactId>flink-yarn_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/flink-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml
index 5946094..e97fee5 100644
--- a/flink-yarn/pom.xml
+++ b/flink-yarn/pom.xml
@@ -27,7 +27,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 	
-	<artifactId>flink-yarn_2.10</artifactId>
+	<artifactId>flink-yarn_${scala.binary.version}</artifactId>
 	<name>flink-yarn</name>
 	<packaging>jar</packaging>
 
@@ -37,7 +37,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<exclusions>
 				<exclusion>
@@ -49,7 +49,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_2.10</artifactId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 
@@ -61,7 +61,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
@@ -98,7 +98,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime_2.10</artifactId>
+			<artifactId>flink-runtime_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/tools/change-scala-version.sh
----------------------------------------------------------------------
diff --git a/tools/change-scala-version.sh b/tools/change-scala-version.sh
deleted file mode 100755
index 56c48c6..0000000
--- a/tools/change-scala-version.sh
+++ /dev/null
@@ -1,117 +0,0 @@
-#!/usr/bin/env bash
-
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# This shell script is from Apache Spark with some modification.
-
-set -e
-
-VALID_VERSIONS=( 2.10 2.11 )
-
-usage() {
-  echo "Usage: $(basename $0) [-h|--help] <scala version to be used>
-where :
-  -h| --help Display this help text
-  valid scala version values : ${VALID_VERSIONS[*]}
-" 1>&2
-  exit 1
-}
-
-if [[ ($# -ne 1) || ( $1 == "--help") ||  $1 == "-h" ]]; then
-  usage
-fi
-
-TO_VERSION=$1
-
-check_scala_version() {
-  for i in ${VALID_VERSIONS[*]}; do [ $i = "$1" ] && return 0; done
-  echo "Invalid Scala version: $1. Valid versions: ${VALID_VERSIONS[*]}" 1>&2
-  exit 1
-}
-
-check_scala_version "$TO_VERSION"
-
-if [ $TO_VERSION = "2.11" ]; then
-  FROM_SUFFIX="_2\.10"
-  TO_SUFFIX="_2\.11"
-else
-  FROM_SUFFIX="_2\.11"
-  TO_SUFFIX="_2\.10"
-fi
-
-sed_i() {
-  sed -e "$1" "$2" > "$2.tmp" && mv "$2.tmp" "$2"
-}
-
-export -f sed_i
-
-echo "sed_i 's/\(artifactId>flink.*'$FROM_SUFFIX'\)<\/artifactId>/\1'$TO_SUFFIX'<\/artifactId>/g' {}";
-
-BASEDIR=$(dirname $0)/..
-find "$BASEDIR" -name 'pom.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(artifactId>flink.*\)'$FROM_SUFFIX'<\/artifactId>/\1'$TO_SUFFIX'<\/artifactId>/g' {}" \;
-
-# fix for examples
-find "$BASEDIR/flink-examples/flink-examples-batch" -name 'pom.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<copy file=\".*flink-examples-batch\)'$FROM_SUFFIX'/\1'$TO_SUFFIX'/g' {}" \;
-
-find "$BASEDIR/flink-examples/flink-examples-streaming" -name 'pom.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<copy file=\".*flink-examples-streaming\)'$FROM_SUFFIX'/\1'$TO_SUFFIX'/g' {}" \;
-
-# fix for quickstart
-find "$BASEDIR/flink-quickstart" -name 'pom.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<exclude>org\.apache\.flink:flink-.*\)'$FROM_SUFFIX'<\/exclude>/\1'$TO_SUFFIX'<\/exclude>/g' {}" \;
-
-# fix for flink-dist (bin.xml)
-find "$BASEDIR/flink-dist" -name 'bin.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<source>.*flink-dist\)'$FROM_SUFFIX'/\1'$TO_SUFFIX'/g' {}" \;
-find "$BASEDIR/flink-dist" -name 'bin.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<include>org\.apache\.flink:flink-.*\)'$FROM_SUFFIX'<\/include>/\1'$TO_SUFFIX'<\/include>/g' {}" \;
-
-# fix for flink-dist (opt.xml)
-find "$BASEDIR/flink-dist" -name 'opt.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<source>.*flink-.*\)'$FROM_SUFFIX'/\1'$TO_SUFFIX'/g' {}" \;
-find "$BASEDIR/flink-dist" -name 'opt.xml' -not -path '*target*' -print \
-  -exec bash -c "sed_i 's/\(<destName>flink-.*\)'$FROM_SUFFIX'/\1'$TO_SUFFIX'/g' {}" \;
-
-# fix for shading curator with Scala 2.11
-find "$BASEDIR/flink-runtime" -name 'pom.xml' -not -path '*target*' -print \
-     -exec bash -c "sed_i 's/\(<include>org\.apache\.flink:flink-shaded-curator.*\)'$FROM_SUFFIX'<\/include>/\1'$TO_SUFFIX'<\/include>/g' {}" \;
-
-if [ "$TO_VERSION" == "2.11" ]; then
-  # set the profile activation to !scala-2.11 in parent pom, so that it activates by default
-  bash -c "sed_i 's/<name>scala-2.11<\/name>/<name>!scala-2.11<\/name>/g' $BASEDIR/pom.xml" \;
-  # set the profile activation in all sub modules to scala-2.11 (so that they are disabled by default)
-  find $BASEDIR/flink-* -name 'pom.xml' -not -path '*target*' -print \
-    -exec bash -c "sed_i 's/<name>!scala-2.11<\/name>/<name>scala-2.11<\/name>/g' {}" \;
-
-  # set the name of the shading artifact properly
-  bash -c "sed_i 's/\(shading-artifact.name>flink-shaded[a-z0-9\-]*\)'$FROM_SUFFIX'<\/shading-artifact.name>/\1'$TO_SUFFIX'<\/shading-artifact.name>/g' $BASEDIR/pom.xml" \;
-fi
-
-if [ "$TO_VERSION" == "2.10" ]; then
-  # do the opposite as above
-  bash -c "sed_i 's/<name>!scala-2.11<\/name>/<name>scala-2.11<\/name>/g' $BASEDIR/pom.xml" \;
-  # also for the other files
-  find $BASEDIR/flink-* -name 'pom.xml' -not -path '*target*' -print \
-    -exec bash -c "sed_i 's/<name>scala-2.11<\/name>/<name>!scala-2.11<\/name>/g' {}" \;
-
-  # unset shading artifact name
-  bash -c "sed_i 's/\(shading-artifact.name>flink-shaded[a-z0-9\-]*\)'$FROM_SUFFIX'<\/shading-artifact.name>/\1'$TO_SUFFIX'<\/shading-artifact.name>/g' $BASEDIR/pom.xml" \;
-fi
-

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/tools/create_release_files.sh
----------------------------------------------------------------------
diff --git a/tools/create_release_files.sh b/tools/create_release_files.sh
index f4c4673..40d038c 100755
--- a/tools/create_release_files.sh
+++ b/tools/create_release_files.sh
@@ -216,10 +216,9 @@ make_binary_release() {
 
   # make distribution
   cd "${dir_name}"
-  ./tools/change-scala-version.sh ${SCALA_VERSION}
 
   # enable release profile here (to check for the maven version)
-  $MVN clean package $FLAGS -DskipTests -Prelease -Dgpg.skip
+  $MVN clean package $FLAGS -DskipTests -Prelease,scala-${SCALA_VERSION} -Dgpg.skip
 
   cd flink-dist/target/flink-*-bin/
   tar czf "${dir_name}.tgz" flink-*
@@ -243,15 +242,12 @@ deploy_to_maven() {
   cp ../../deploysettings.xml .
   
   echo "Deploying Scala 2.11 version"
-  cd tools && ./change-scala-version.sh 2.11 && cd ..
-
-  $MVN clean deploy -Prelease,docs-and-source --settings deploysettings.xml -DskipTests -Dgpg.executable=$GPG -Dgpg.keyname=$GPG_KEY -Dgpg.passphrase=$GPG_PASSPHRASE -DretryFailedDeploymentCount=10
+  $MVN clean deploy -Prelease,docs-and-source,scala-2.11 --settings deploysettings.xml -DskipTests -Dgpg.executable=$GPG -Dgpg.keyname=$GPG_KEY -Dgpg.passphrase=$GPG_PASSPHRASE -DretryFailedDeploymentCount=10
   
   # It is important to first deploy scala 2.11 and then scala 2.10 so that the quickstarts (which are independent of the scala version)
   # are depending on scala 2.10.
   echo "Deploying Scala 2.10 version"
-  cd tools && ./change-scala-version.sh 2.10 && cd ..
-  $MVN clean deploy -Dgpg.executable=$GPG -Prelease,docs-and-source --settings deploysettings.xml -DskipTests -Dgpg.keyname=$GPG_KEY -Dgpg.passphrase=$GPG_PASSPHRASE -DretryFailedDeploymentCount=10
+  $MVN clean deploy -Prelease,docs-and-source,scala-2.10 --settings deploysettings.xml -DskipTests -Dgpg.executable=$GPG -Dgpg.keyname=$GPG_KEY -Dgpg.passphrase=$GPG_PASSPHRASE -DretryFailedDeploymentCount=10
 }
 
 copy_data() {

http://git-wip-us.apache.org/repos/asf/flink/blob/3990d75a/tools/deploy_to_maven.sh
----------------------------------------------------------------------
diff --git a/tools/deploy_to_maven.sh b/tools/deploy_to_maven.sh
index 16d529e..32acc5a 100755
--- a/tools/deploy_to_maven.sh
+++ b/tools/deploy_to_maven.sh
@@ -89,18 +89,14 @@ if [[ $CURRENT_FLINK_VERSION == *SNAPSHOT* ]] ; then
 
     # hadoop2 scala 2.10
     echo "deploy standard version (hadoop2) for scala 2.10"
-    mvn ${MVN_SNAPSHOT_OPTS}
+    mvn ${MVN_SNAPSHOT_OPTS} -Pscala-2.10
     deploy_to_s3 $CURRENT_FLINK_VERSION "hadoop2"
 
     # hadoop2 scala 2.11
     echo "deploy hadoop2 version (standard) for scala 2.11"
-    ./tools/change-scala-version.sh 2.11
-    mvn ${MVN_SNAPSHOT_OPTS}
+    mvn ${MVN_SNAPSHOT_OPTS} -Pscala-2.11
     deploy_to_s3 $CURRENT_FLINK_VERSION "hadoop2_2.11"
 
-    echo "Changing back to scala 2.10"
-    ./tools/change-scala-version.sh 2.10
-
     exit 0
 else
     exit 1


[05/14] flink git commit: [FLINK-6527] [checkpoint] OperatorSubtaskState has empty implementations of (un)/registerSharedStates

Posted by sr...@apache.org.
[FLINK-6527] [checkpoint] OperatorSubtaskState has empty implementations of (un)/registerSharedStates


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

Branch: refs/heads/release-1.3
Commit: 37e3deae91070b9ecce5ae65be767a4c732a60d4
Parents: f7d79d8
Author: Stefan Richter <s....@data-artisans.com>
Authored: Wed May 10 14:57:55 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../runtime/checkpoint/OperatorSubtaskState.java    | 16 ++++++++++++++--
 1 file changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/37e3deae/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
index 863816a..49ef863 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
@@ -148,12 +148,24 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 
 	@Override
 	public void registerSharedStates(SharedStateRegistry sharedStateRegistry) {
-		// No shared states
+		if (managedKeyedState != null) {
+			managedKeyedState.registerSharedStates(sharedStateRegistry);
+		}
+
+		if (rawKeyedState != null) {
+			rawKeyedState.registerSharedStates(sharedStateRegistry);
+		}
 	}
 
 	@Override
 	public void unregisterSharedStates(SharedStateRegistry sharedStateRegistry) {
-		// No shared states
+		if (managedKeyedState != null) {
+			managedKeyedState.unregisterSharedStates(sharedStateRegistry);
+		}
+
+		if (rawKeyedState != null) {
+			rawKeyedState.unregisterSharedStates(sharedStateRegistry);
+		}
 	}
 
 	@Override


[02/14] flink git commit: [FLINK-5978] Move JM WebFrontend address ConfigOption to JMOptions

Posted by sr...@apache.org.
[FLINK-5978] Move JM WebFrontend address ConfigOption to JMOptions

This closes #3552.


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

Branch: refs/heads/release-1.3
Commit: 60deaef08a8aa2a1a1cdc98acf45c066e96cd1ad
Parents: fa49915
Author: mengji.fy <me...@taobao.com>
Authored: Fri May 5 12:11:20 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:06:28 2017 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/configuration/ConfigConstants.java  | 7 ++++++-
 .../org/apache/flink/configuration/JobManagerOptions.java     | 7 +++++++
 .../org/apache/flink/runtime/webmonitor/WebMonitorConfig.java | 2 +-
 3 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/60deaef0/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 61c1b27..92e6b5d 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -1328,7 +1328,12 @@ public final class ConfigConstants {
 	
 	// ------------------------- JobManager Web Frontend ----------------------
 
-	/** The config key for the address of the JobManager web frontend. */
+	/**
+	 * The config key for the address of the JobManager web frontend.
+	 *
+	 * @deprecated use {@link JobManagerOptions#WEB_FRONTEND_ADDRESS} instead
+	 */
+	@Deprecated
 	public static final ConfigOption<String> DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESS =
 		key("jobmanager.web.address")
 			.noDefaultValue();

http://git-wip-us.apache.org/repos/asf/flink/blob/60deaef0/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
index 140ba2e..b924e8e 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
@@ -94,6 +94,13 @@ public class JobManagerOptions {
 	// ------------------------------------------------------------------------
 
 	/**
+	 * Config parameter defining the runtime monitor web-frontend server address.
+	 */
+	public static final ConfigOption<String> WEB_FRONTEND_ADDRESS =
+		key("jobmanager.web.address")
+			.noDefaultValue();
+
+	/**
 	 * The port for the runtime monitor web-frontend server.
 	 */
 	public static final ConfigOption<Integer> WEB_PORT =

http://git-wip-us.apache.org/repos/asf/flink/blob/60deaef0/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
index d95d13a..dba2145 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorConfig.java
@@ -63,7 +63,7 @@ public class WebMonitorConfig {
 	}
 
 	public String getWebFrontendAddress() {
-		return config.getValue(ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESS);
+		return config.getValue(JobManagerOptions.WEB_FRONTEND_ADDRESS);
 	}
 
 	public int getWebFrontendPort() {


[07/14] flink git commit: [FLINK-6509] [tests] Perform TestingListener#waitForNewLeader under lock

Posted by sr...@apache.org.
[FLINK-6509] [tests] Perform TestingListener#waitForNewLeader under lock

Performin TestingListener#waitForNewLeader under the lock which is also hold when
updating the leader information makes sure that leader changes won't go unnoticed.
This led before to failing test cases due to timeouts.

This closes #3853.


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

Branch: refs/heads/release-1.3
Commit: d88a62f9c0030a35ffdec5c252c8428ab8960521
Parents: 9708550
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue May 9 13:13:02 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/leaderelection/TestingListener.java | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d88a62f9/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
index 87decc7..8571505 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
@@ -51,11 +51,11 @@ public class TestingListener implements LeaderRetrievalListener {
 		long start = System.currentTimeMillis();
 		long curTimeout;
 
-		while (
+		synchronized (lock) {
+			while (
 				exception == null &&
-				(address == null || address.equals(oldAddress)) &&
-				(curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
-			synchronized (lock) {
+					(address == null || address.equals(oldAddress)) &&
+					(curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
 				try {
 					lock.wait(curTimeout);
 				} catch (InterruptedException e) {


[04/14] flink git commit: [hotfix] Minutiae

Posted by sr...@apache.org.
[hotfix] Minutiae


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

Branch: refs/heads/release-1.3
Commit: 9fe61355004164cf41a3e01a96218c11ea876daa
Parents: 104ea24
Author: Greg Hogan <co...@greghogan.com>
Authored: Thu Feb 2 13:23:04 2017 -0500
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/CliFrontend.java    |  2 +-
 .../api/io/avro/AvroRecordInputFormatTest.java  | 26 +++++++----
 .../connectors/fs/SequenceFileWriter.java       |  2 +-
 .../common/accumulators/AccumulatorHelper.java  |  2 +-
 .../flink/api/java/typeutils/TypeExtractor.java |  2 +-
 .../AbstractGenericArraySerializerTest.java     | 16 +++----
 .../AbstractGenericTypeComparatorTest.java      |  6 +--
 .../AbstractGenericTypeSerializerTest.java      |  2 +-
 .../typeutils/runtime/kryo/SerializersTest.java |  2 +-
 .../apache/flink/types/BasicTypeInfoTest.java   |  9 ++--
 .../flink/api/java/operators/Grouping.java      |  2 +-
 .../java/functions/SelectByFunctionsTest.java   | 18 ++++----
 .../java/functions/SemanticPropUtilTest.java    | 32 ++++++-------
 .../traversals/GraphCreatingVisitor.java        |  4 +-
 .../flink/runtime/util/SerializedThrowable.java | 10 ++---
 .../scheduler/SchedulerSlotSharingTest.java     |  2 +-
 .../api/operator/SelectByFunctionTest.scala     | 18 ++++----
 .../operators/AbstractStreamOperatorTest.java   | 47 ++++++++++----------
 .../runtime/io/BarrierTrackerTest.java          |  2 +-
 .../test/accumulators/AccumulatorITCase.java    |  2 +-
 20 files changed, 107 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index 74d5f5d..62fa402 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -860,7 +860,7 @@ public class CliFrontend {
 			Map<String, Object> accumulatorsResult = execResult.getAllAccumulatorResults();
 			if (accumulatorsResult.size() > 0) {
 				System.out.println("Accumulator Results: ");
-				System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
+				System.out.println(AccumulatorHelper.getResultsFormatted(accumulatorsResult));
 			}
 		} else {
 			logAndSysout("Job has been submitted with JobID " + result.getJobID());

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
index 91a9612..3b6ad63 100644
--- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroRecordInputFormatTest.java
@@ -18,14 +18,6 @@
 
 package org.apache.flink.api.io.avro;
 
-import static org.junit.Assert.*;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileReader;
 import org.apache.avro.file.DataFileWriter;
@@ -59,6 +51,22 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Test the avro input format.
  * (The testcase is mostly the getting started tutorial of avro)
@@ -273,7 +281,7 @@ public class AvroRecordInputFormatTest {
 		DatumReader<GenericData.Record> datumReader = new GenericDatumReader<>(userSchema);
 
 		try (FileReader<GenericData.Record> dataFileReader = DataFileReader.openReader(testFile, datumReader)) {
-			// initialize Record by reading it from disk (thats easier than creating it by hand)
+			// initialize Record by reading it from disk (that's easier than creating it by hand)
 			GenericData.Record rec = new GenericData.Record(userSchema);
 			dataFileReader.next(rec);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
index 08c0d0a..32cadec 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
@@ -38,7 +38,7 @@ import java.io.IOException;
 /**
  * A {@link Writer} that writes the bucket files as Hadoop {@link SequenceFile SequenceFiles}.
  * The input to the {@link BucketingSink} must
- * be a {@link org.apache.flink.api.java.tuple.Tuple2} of two Hadopo
+ * be a {@link org.apache.flink.api.java.tuple.Tuple2} of two Hadoop
  * {@link org.apache.hadoop.io.Writable Writables}.
  *
  * @param <K> The type of the first tuple field.

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java
index 1a87235..3282302 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java
@@ -112,7 +112,7 @@ public class AccumulatorHelper {
 		return resultMap;
 	}
 
-	public static String getResultsFormated(Map<String, Object> map) {
+	public static String getResultsFormatted(Map<String, Object> map) {
 		StringBuilder builder = new StringBuilder();
 		for (Map.Entry<String, Object> entry : map.entrySet()) {
 			builder

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index f1bf957..112ca38 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -394,7 +394,7 @@ public class TypeExtractor {
 	 * @param inputTypeArgumentIndex Index of the type argument of function's first parameter
 	 *                               specifying the input type if it is wrapped (Iterable, Map,
 	 *                               etc.). Otherwise -1.
-	 * @param outputTypeArgumentIndex Index of the type argument of functions second parameter
+	 * @param outputTypeArgumentIndex Index of the type argument of function's second parameter
 	 *                                specifying the output type if it is wrapped in a Collector.
 	 *                                Otherwise -1.
 	 * @param inType Type of the input elements (In case of an iterable, it is the element type)

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java
index 307a3d2..7386e6e 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
-import java.lang.reflect.Array;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.junit.Assert;
-import org.junit.Test;
 import org.apache.flink.api.common.typeutils.SerializerTestInstance;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.GenericArraySerializer;
@@ -34,6 +28,12 @@ import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeSerializer
 import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeSerializerTest.ComplexNestedObject2;
 import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeSerializerTest.SimpleTypes;
 import org.apache.flink.util.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Random;
 
 public abstract class AbstractGenericArraySerializerTest {
 	
@@ -110,11 +110,11 @@ public abstract class AbstractGenericArraySerializerTest {
 	@Test
 	public void testBeanStyleObjects() {
 		{
-			Book b1 = new Book(976243875L, "The Serialization Odysse", 42);
+			Book b1 = new Book(976243875L, "The Serialization Odyssey", 42);
 			Book b2 = new Book(0L, "Debugging byte streams", 1337);
 			Book b3 = new Book(-1L, "Low level interfaces", 0xC0FFEE);
 			Book b4 = new Book(Long.MAX_VALUE, "The joy of bits and bytes", 0xDEADBEEF);
-			Book b5 = new Book(Long.MIN_VALUE, "Winnign a prize for creative test strings", 0xBADF00);
+			Book b5 = new Book(Long.MIN_VALUE, "Winning a prize for creative test strings", 0xBADF00);
 			Book b6 = new Book(-2L, "Distributed Systems", 0xABCDEF0123456789L);
 			
 			runTests(	new Book[] {b1, b2},

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java
index ef168d8..44464a3 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java
@@ -30,16 +30,14 @@ abstract public class AbstractGenericTypeComparatorTest {
 
 	@Test
 	public void testString() {
-		runTests(new String[]{
-				"",
+		runTests("",
 				"Lorem Ipsum Dolor Omit Longer",
 				"aaaa",
 				"abcd",
 				"abce",
 				"abdd",
 				"accd",
-				"bbcd"
-		});
+				"bbcd");
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java
index 059c78d..a75208a 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java
@@ -88,7 +88,7 @@ abstract public class AbstractGenericTypeSerializerTest {
 	@Test
 	public void testBeanStyleObjects() {
 		{
-			Book b1 = new Book(976243875L, "The Serialization Odysse", 42);
+			Book b1 = new Book(976243875L, "The Serialization Odyssey", 42);
 			Book b2 = new Book(0L, "Debugging byte streams", 1337);
 			Book b3 = new Book(-1L, "Low level interfaces", 0xC0FFEE);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java
index 7c6d023..4a39f8f 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java
@@ -46,7 +46,7 @@ public class SerializersTest {
 	
 	public static class Nested1 {
 		private FromNested fromNested;
-		private Path yodaIntervall;
+		private Path yodaInterval;
 	}
 
 	public static class ClassWithNested {

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
index c090b76..5707701 100644
--- a/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
@@ -11,22 +11,23 @@
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WNTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
 
 package org.apache.flink.types;
 
-import java.math.BigDecimal;
-import java.math.BigInteger;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.util.Date;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 
 public class BasicTypeInfoTest extends TestLogger {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
index 8601c33..dbaaa9d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
@@ -29,7 +29,7 @@ import org.apache.flink.api.java.DataSet;
  * Grouping is an intermediate step for a transformation on a grouped DataSet.<br>
  * The following transformation can be applied on Grouping:
  * <ul>
- * 	<li>{@link UnsortedGrouping#reduce(org.apache.flink.api.common.functions.ReduceFunction)},</li>
+ * <li>{@link UnsortedGrouping#reduce(org.apache.flink.api.common.functions.ReduceFunction)},</li>
  * <li>{@link UnsortedGrouping#reduceGroup(org.apache.flink.api.common.functions.GroupReduceFunction)}, and</li>
  * <li>{@link UnsortedGrouping#aggregate(org.apache.flink.api.java.aggregation.Aggregations, int)}.</li>
  * </ul>

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java
index f40c0ca..52b59ec 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java
@@ -49,7 +49,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(smaller, bigger));
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(bigger, smaller));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -67,7 +67,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMax must return the first given tuple", specialCaseBigger, maxByTuple.reduce(specialCaseBigger, bigger));
 			Assert.assertSame("SelectByMax must return the first given tuple", bigger, maxByTuple.reduce(bigger, specialCaseBigger));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -82,7 +82,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(specialCaseBigger, bigger));
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(bigger, specialCaseBigger));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -97,7 +97,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(smaller, bigger));
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(bigger, smaller));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -112,7 +112,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMax must return bigger tuple", bigger, maxByTuple.reduce(bigger, bigger));
 			Assert.assertSame("SelectByMax must return smaller tuple", smaller, maxByTuple.reduce(smaller, smaller));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -129,7 +129,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMin must return smaller tuple", smaller, minByTuple.reduce(smaller, bigger));
 			Assert.assertSame("SelectByMin must return smaller tuple", smaller, minByTuple.reduce(bigger, smaller));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -145,7 +145,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMin must return the first given tuple", specialCaseBigger, minByTuple.reduce(specialCaseBigger, bigger));
 			Assert.assertSame("SelectByMin must return the first given tuple", bigger, minByTuple.reduce(bigger, specialCaseBigger));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -161,7 +161,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMin must return smaller tuple", smaller, minByTuple.reduce(specialCaseSmaller, smaller));
 			Assert.assertSame("SelectByMin must return smaller tuple", smaller, minByTuple.reduce(smaller, specialCaseSmaller));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	
@@ -176,7 +176,7 @@ public class SelectByFunctionsTest {
 			Assert.assertSame("SelectByMin must return smaller tuple", smaller, minByTuple.reduce(smaller, bigger));
 			Assert.assertSame("SelectByMin must return smaller tuple", smaller, minByTuple.reduce(bigger, smaller));
 		} catch (Exception e) {
-			Assert.fail("No exception should be thrown while comapring both tuples");
+			Assert.fail("No exception should be thrown while comparing both tuples");
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
index b845e73..453c022 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
@@ -324,7 +324,7 @@ public class SemanticPropUtilTest {
 	// --------------------------------------------------------------------------------------------
 
 	@Test
-	public void testForwardedNoArrrowIndividualStrings() {
+	public void testForwardedNoArrowIndividualStrings() {
 		String[] forwardedFields = {"f2","f3","f0"};
 		SingleInputSemanticProperties sp = new SingleInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsSingleFromString(sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType);
@@ -335,7 +335,7 @@ public class SemanticPropUtilTest {
 	}
 
 	@Test
-	public void testForwardedNoArrrowOneString() {
+	public void testForwardedNoArrowOneString() {
 		String[] forwardedFields = {"f2;f3;f0"};
 		SingleInputSemanticProperties sp = new SingleInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsSingleFromString(sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType);
@@ -360,7 +360,7 @@ public class SemanticPropUtilTest {
 	}
 
 	@Test
-	public void testForwardedNoArrrowSpaces() {
+	public void testForwardedNoArrowSpaces() {
 		String[] forwardedFields = {"  f2  ;   f3  ;  f0   "};
 		SingleInputSemanticProperties sp = new SingleInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsSingleFromString(sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType);
@@ -1219,11 +1219,11 @@ public class SemanticPropUtilTest {
 
 	@Test
 	public void testNonForwardedDual() {
-		String[] nonNorwardedFieldsFirst = { "f1;f2" };
-		String[] nonNorwardedFieldsSecond = { "f0" };
+		String[] nonForwardedFieldsFirst = { "f1;f2" };
+		String[] nonForwardedFieldsSecond = { "f0" };
 		DualInputSemanticProperties dsp = new DualInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsDualFromString(dsp, null, null,
-				nonNorwardedFieldsFirst, nonNorwardedFieldsSecond, null, null, threeIntTupleType, threeIntTupleType, threeIntTupleType);
+				nonForwardedFieldsFirst, nonForwardedFieldsSecond, null, null, threeIntTupleType, threeIntTupleType, threeIntTupleType);
 
 		assertTrue(dsp.getForwardingTargetFields(0, 0).contains(0));
 		assertTrue(dsp.getForwardingTargetFields(0, 1).size() == 0);
@@ -1232,11 +1232,11 @@ public class SemanticPropUtilTest {
 		assertTrue(dsp.getForwardingTargetFields(1, 1).contains(1));
 		assertTrue(dsp.getForwardingTargetFields(1, 2).contains(2));
 
-		nonNorwardedFieldsFirst[0] = "f1";
-		nonNorwardedFieldsSecond[0] = "";
+		nonForwardedFieldsFirst[0] = "f1";
+		nonForwardedFieldsSecond[0] = "";
 		dsp = new DualInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsDualFromString(dsp, null, null,
-				nonNorwardedFieldsFirst, null, null, null, threeIntTupleType, fiveIntTupleType, threeIntTupleType);
+				nonForwardedFieldsFirst, null, null, null, threeIntTupleType, fiveIntTupleType, threeIntTupleType);
 
 		assertTrue(dsp.getForwardingTargetFields(0, 0).contains(0));
 		assertTrue(dsp.getForwardingTargetFields(0, 1).size() == 0);
@@ -1245,11 +1245,11 @@ public class SemanticPropUtilTest {
 		assertTrue(dsp.getForwardingTargetFields(1, 1).size() == 0);
 		assertTrue(dsp.getForwardingTargetFields(1, 2).size() == 0);
 
-		nonNorwardedFieldsFirst[0] = "";
-		nonNorwardedFieldsSecond[0] = "f2;f0";
+		nonForwardedFieldsFirst[0] = "";
+		nonForwardedFieldsSecond[0] = "f2;f0";
 		dsp = new DualInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsDualFromString(dsp, null, null,
-				null, nonNorwardedFieldsSecond, null, null, fiveIntTupleType, threeIntTupleType, threeIntTupleType);
+				null, nonForwardedFieldsSecond, null, null, fiveIntTupleType, threeIntTupleType, threeIntTupleType);
 
 		assertTrue(dsp.getForwardingTargetFields(0, 0).size() == 0);
 		assertTrue(dsp.getForwardingTargetFields(0, 1).size() == 0);
@@ -1283,19 +1283,19 @@ public class SemanticPropUtilTest {
 	@Test(expected = InvalidSemanticAnnotationException.class)
 	public void testNonForwardedDualInvalidTypes1() {
 
-		String[] nonNorwardedFieldsFirst = { "f1" };
+		String[] nonForwardedFieldsFirst = { "f1" };
 		DualInputSemanticProperties dsp = new DualInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsDualFromString(dsp, null, null,
-				nonNorwardedFieldsFirst, null, null, null, fiveIntTupleType, threeIntTupleType, threeIntTupleType);
+				nonForwardedFieldsFirst, null, null, null, fiveIntTupleType, threeIntTupleType, threeIntTupleType);
 	}
 
 	@Test(expected = InvalidSemanticAnnotationException.class)
 	public void testNonForwardedDualInvalidTypes2() {
 
-		String[] nonNorwardedFieldsSecond = { "f1" };
+		String[] nonForwardedFieldsSecond = { "f1" };
 		DualInputSemanticProperties dsp = new DualInputSemanticProperties();
 		SemanticPropUtil.getSemanticPropsDualFromString(dsp, null, null,
-				null, nonNorwardedFieldsSecond, null, null, threeIntTupleType, pojoInTupleType, threeIntTupleType);
+				null, nonForwardedFieldsSecond, null, null, threeIntTupleType, pojoInTupleType, threeIntTupleType);
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
index 3f3eae1..92217b5 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
@@ -76,8 +76,8 @@ import java.util.Map;
 /**
  * This traversal creates the optimizer DAG from a program.
  * It works as a visitor that walks the program's flow in a depth-first fashion, starting from the data sinks.
- * During the descend, it creates an optimizer node for each operator, respectively data source or -sink.
- * During the ascend, it connects the nodes to the full graph.
+ * During the descent it creates an optimizer node for each operator, respectively data source or sink.
+ * During the ascent it connects the nodes to the full graph.
  */
 public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
index f6f08e4..63f4363 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
@@ -48,7 +48,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 	private final String originalErrorClassName;
 	
 	/** The original stack trace, to be printed */
-	private final String fullStingifiedStackTrace;
+	private final String fullStringifiedStackTrace;
 
 	/** The original exception, not transported via serialization, 
 	 * because the class may not be part of the system class loader.
@@ -83,7 +83,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 
 			// record the original exception's properties (name, stack prints)
 			this.originalErrorClassName = exception.getClass().getName();
-			this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception);
+			this.fullStringifiedStackTrace = ExceptionUtils.stringifyException(exception);
 
 			// mimic the original exception's stack trace
 			setStackTrace(exception.getStackTrace());
@@ -106,7 +106,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 			SerializedThrowable other = (SerializedThrowable) exception;
 			this.serializedException = other.serializedException;
 			this.originalErrorClassName = other.originalErrorClassName;
-			this.fullStingifiedStackTrace = other.fullStingifiedStackTrace;
+			this.fullStringifiedStackTrace = other.fullStringifiedStackTrace;
 			this.cachedException = other.cachedException;
 			this.setStackTrace(other.getStackTrace());
 			this.initCause(other.getCause());
@@ -145,13 +145,13 @@ public class SerializedThrowable extends Exception implements Serializable {
 
 	@Override
 	public void printStackTrace(PrintStream s) {
-		s.print(fullStingifiedStackTrace);
+		s.print(fullStringifiedStackTrace);
 		s.flush();
 	}
 	
 	@Override
 	public void printStackTrace(PrintWriter s) {
-		s.print(fullStingifiedStackTrace);
+		s.print(fullStringifiedStackTrace);
 		s.flush();
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
index 5238e95..c049593 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
@@ -320,7 +320,7 @@ public class SchedulerSlotSharingTest {
 	}
 	
 	@Test
-	public void allocateSlotWithTemprarilyEmptyVertexGroup() {
+	public void allocateSlotWithTemporarilyEmptyVertexGroup() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-scala/src/test/scala/org/apache/flink/api/operator/SelectByFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/operator/SelectByFunctionTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/operator/SelectByFunctionTest.scala
index 291df79..d6af6bd 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/operator/SelectByFunctionTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/operator/SelectByFunctionTest.scala
@@ -54,7 +54,7 @@ class SelectByFunctionTest {
           bigger, maxByTuple.reduce(bigger, smaller))
       } catch {
         case e : Exception =>
-          Assert.fail("No exception should be thrown while comapring both tuples")
+          Assert.fail("No exception should be thrown while comparing both tuples")
       }
   }
 
@@ -77,7 +77,7 @@ class SelectByFunctionTest {
         bigger, maxByTuple.reduce(bigger, specialCaseBigger))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown " +
-        "while comapring both tuples")
+        "while comparing both tuples")
     }
   }
 
@@ -95,7 +95,7 @@ class SelectByFunctionTest {
         bigger, maxByTuple.reduce(bigger, specialCaseBigger))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown" +
-        " while comapring both tuples")
+        " while comparing both tuples")
     }
   }
 
@@ -113,7 +113,7 @@ class SelectByFunctionTest {
         bigger, maxByTuple.reduce(bigger, smaller))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown " +
-        "while comapring both tuples")
+        "while comparing both tuples")
     }
   }
 
@@ -132,7 +132,7 @@ class SelectByFunctionTest {
         smaller, maxByTuple.reduce(smaller, smaller))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown" +
-        " while comapring both tuples")
+        " while comparing both tuples")
     }
   }
 
@@ -153,7 +153,7 @@ class SelectByFunctionTest {
         smaller, minByTuple.reduce(bigger, smaller))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown " +
-        "while comapring both tuples")
+        "while comparing both tuples")
     }
   }
 
@@ -174,7 +174,7 @@ class SelectByFunctionTest {
         bigger, minByTuple.reduce(bigger, specialCaseBigger))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown " +
-        "while comapring both tuples")
+        "while comparing both tuples")
     }
   }
 
@@ -195,7 +195,7 @@ class SelectByFunctionTest {
         smaller, minByTuple.reduce(smaller, specialCaseSmaller))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown" +
-        " while comapring both tuples")
+        " while comparing both tuples")
     }
   }
 
@@ -213,7 +213,7 @@ class SelectByFunctionTest {
         smaller, minByTuple.reduce(bigger, smaller))
     } catch {
       case e : Exception => Assert.fail("No exception should be thrown" +
-        " while comapring both tuples")
+        " while comparing both tuples")
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
index 774ab4a..67004ea 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
@@ -17,28 +17,6 @@
  */
 package org.apache.flink.streaming.api.operators;
 
-import static junit.framework.TestCase.assertTrue;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.verify;
-import static org.powermock.api.mockito.PowerMockito.doReturn;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.spy;
-import static org.powermock.api.mockito.PowerMockito.when;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.RunnableFuture;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
@@ -71,6 +49,29 @@ import org.mockito.internal.util.reflection.Whitebox;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.RunnableFuture;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.verify;
+import static org.powermock.api.mockito.PowerMockito.doReturn;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.spy;
+import static org.powermock.api.mockito.PowerMockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
 /**
  * Tests for the facilities provided by {@link AbstractStreamOperator}. This mostly
  * tests timers and state and whether they are correctly checkpointed/restored
@@ -510,7 +511,7 @@ public class AbstractStreamOperatorTest {
 
 	/**
 	 * Tests that the created StateSnapshotContextSynchronousImpl is closed in case of a failing
-	 * Operator#snapshotState(StaetSnapshotContextSynchronousImpl) call.
+	 * Operator#snapshotState(StateSnapshotContextSynchronousImpl) call.
 	 */
 	@Test
 	public void testFailingSnapshotMethod() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index cbcf45d..8c66205 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -526,7 +526,7 @@ public class BarrierTrackerTest {
 
 			final long expectedId = checkpointIDs[i++];
 			if (expectedId < 0) {
-				assertEquals("wrong checkpoint id for checkoint abort", -expectedId, checkpointId);
+				assertEquals("wrong checkpoint id for checkpoint abort", -expectedId, checkpointId);
 			} else {
 				fail("got 'abortCheckpointOnBarrier()' when expecting an 'triggerCheckpointOnBarrier()'");
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9fe61355/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
index b4015e5..5f2b0a9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
@@ -74,7 +74,7 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 		// Test accumulator results
 		System.out.println("Accumulator results:");
 		JobExecutionResult res = this.result;
-		System.out.println(AccumulatorHelper.getResultsFormated(res.getAllAccumulatorResults()));
+		System.out.println(AccumulatorHelper.getResultsFormatted(res.getAllAccumulatorResults()));
 
 		Assert.assertEquals(Integer.valueOf(3), (Integer) res.getAccumulatorResult("num-lines"));
 


[11/14] flink git commit: [hotfix] [build] Drop transitive jersey/jettison/servlet dependencies pulled via Hadoop

Posted by sr...@apache.org.
[hotfix] [build] Drop transitive jersey/jettison/servlet dependencies pulled via Hadoop


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

Branch: refs/heads/release-1.3
Commit: 104ea244e94aa8be7bd5089c13cd8292905eb6fd
Parents: 3990d75
Author: Stephan Ewen <se...@apache.org>
Authored: Thu May 11 15:12:10 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../flink-shaded-hadoop2/pom.xml                | 61 ++++++++++++++++----
 1 file changed, 50 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/104ea244/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
index 9ac033e..a1159d6 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
@@ -70,20 +70,23 @@ under the License.
 				</exclusion>
 				<exclusion>
 					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>jsp-api-2.1</artifactId>
 				</exclusion>
 				<exclusion>
 					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>jsp-2.1</artifactId>
 				</exclusion>
-
 				<exclusion>
 					<groupId>org.eclipse.jdt</groupId>
 					<artifactId>core</artifactId>
 				</exclusion>
 				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-core</artifactId>
 				</exclusion>
 				<exclusion>
 					<groupId>com.sun.jersey</groupId>
@@ -106,6 +109,10 @@ under the License.
 					<artifactId>jasper-runtime</artifactId>
 				</exclusion>
 				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
 					<groupId>javax.servlet.jsp</groupId>
 					<artifactId>jsp-api</artifactId>
 				</exclusion>
@@ -191,6 +198,10 @@ under the License.
 				</exclusion>
 				<exclusion>
 					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>jsp-api-2.1</artifactId>
 				</exclusion>
 				<exclusion>
@@ -202,8 +213,8 @@ under the License.
 					<artifactId>core</artifactId>
 				</exclusion>
 				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-core</artifactId>
 				</exclusion>
 				<exclusion>
 					<groupId>com.sun.jersey</groupId>
@@ -226,6 +237,10 @@ under the License.
 					<artifactId>jasper-runtime</artifactId>
 				</exclusion>
 				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
 					<groupId>javax.servlet.jsp</groupId>
 					<artifactId>jsp-api</artifactId>
 				</exclusion>
@@ -299,6 +314,10 @@ under the License.
 				</exclusion>
 				<exclusion>
 					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>jsp-api-2.1</artifactId>
 				</exclusion>
 				<exclusion>
@@ -322,8 +341,8 @@ under the License.
 					<artifactId>netty</artifactId>
 				</exclusion>
 				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-core</artifactId>
 				</exclusion>
 				<exclusion>
 					<groupId>com.sun.jersey</groupId>
@@ -346,6 +365,10 @@ under the License.
 					<artifactId>jasper-runtime</artifactId>
 				</exclusion>
 				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
 					<groupId>javax.servlet.jsp</groupId>
 					<artifactId>jsp-api</artifactId>
 				</exclusion>
@@ -419,6 +442,10 @@ under the License.
 				</exclusion>
 				<exclusion>
 					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>jsp-api-2.1</artifactId>
 				</exclusion>
 				<exclusion>
@@ -442,8 +469,8 @@ under the License.
 					<artifactId>netty</artifactId>
 				</exclusion>
 				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-core</artifactId>
 				</exclusion>
 				<exclusion>
 					<groupId>com.sun.jersey</groupId>
@@ -466,6 +493,10 @@ under the License.
 					<artifactId>jasper-runtime</artifactId>
 				</exclusion>
 				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
 					<groupId>javax.servlet.jsp</groupId>
 					<artifactId>jsp-api</artifactId>
 				</exclusion>
@@ -539,6 +570,10 @@ under the License.
 				</exclusion>
 				<exclusion>
 					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>jsp-api-2.1</artifactId>
 				</exclusion>
 				<exclusion>
@@ -562,8 +597,8 @@ under the License.
 					<artifactId>netty</artifactId>
 				</exclusion>
 				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
+					<groupId>com.sun.jersey</groupId>
+					<artifactId>jersey-core</artifactId>
 				</exclusion>
 				<exclusion>
 					<groupId>com.sun.jersey</groupId>
@@ -586,6 +621,10 @@ under the License.
 					<artifactId>jasper-runtime</artifactId>
 				</exclusion>
 				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
 					<groupId>javax.servlet.jsp</groupId>
 					<artifactId>jsp-api</artifactId>
 				</exclusion>


[10/14] flink git commit: [FLINK-6518] Port blobserver config parameters to ConfigOptions

Posted by sr...@apache.org.
[FLINK-6518] Port blobserver config parameters to ConfigOptions

This closes #3865.


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

Branch: refs/heads/release-1.3
Commit: 5e61a01bae6d05bf1d5c76bc48f0ba90bbdef752
Parents: 9fe6135
Author: zentol <ch...@apache.org>
Authored: Wed May 10 10:26:19 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../flink/configuration/BlobServerOptions.java  | 76 ++++++++++++++++++++
 .../flink/configuration/ConfigConstants.java    | 40 ++++++-----
 .../apache/flink/runtime/blob/BlobCache.java    |  9 ++-
 .../apache/flink/runtime/blob/BlobClient.java   |  5 +-
 .../apache/flink/runtime/blob/BlobServer.java   | 22 +++---
 .../flink/runtime/blob/BlobClientSslTest.java   |  5 +-
 .../flink/runtime/blob/BlobServerRangeTest.java |  8 +--
 .../jobmanager/JobManagerStartupTest.java       |  4 +-
 8 files changed, 125 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
new file mode 100644
index 0000000..e27c29f
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * Configuration options for the BlobServer.
+ */
+@PublicEvolving
+public class BlobServerOptions {
+
+	/**
+	 * The config parameter defining the storage directory to be used by the blob server.
+	 */
+	public static final ConfigOption<String> STORAGE_DIRECTORY =
+		key("blob.storage.directory")
+			.noDefaultValue();
+
+	/**
+	 * The config parameter defining number of retires for failed BLOB fetches.
+	 */
+	public static final ConfigOption<Integer> FETCH_RETRIES =
+		key("blob.fetch.retries")
+			.defaultValue(5);
+
+	/**
+	 * The config parameter defining the maximum number of concurrent BLOB fetches that the JobManager serves.
+	 */
+	public static final ConfigOption<Integer> FETCH_CONCURRENT =
+		key("blob.fetch.num-concurrent")
+			.defaultValue(50);
+
+	/**
+	 * The config parameter defining the backlog of BLOB fetches on the JobManager.
+	 */
+	public static final ConfigOption<Integer> FETCH_BACKLOG =
+		key("blob.fetch.backlog")
+			.defaultValue(1000);
+
+	/**
+	 * The config parameter defining the server port of the blob service.
+	 * The port can either be a port, such as "9123",
+	 * a range of ports: "50100-50200"
+	 * or a list of ranges and or points: "50100-50200,50300-50400,51234"
+	 *
+	 * Setting the port to 0 will let the OS choose an available port.
+	 */
+	public static final ConfigOption<String> PORT =
+		key("blob.server.port")
+			.defaultValue("0");
+
+	/**
+	 * Flag to override ssl support for the blob service transport.
+	 */
+	public static final ConfigOption<Boolean> SSL_ENABLED =
+		key("blob.service.ssl.enabled")
+			.defaultValue(true);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index c3704be..b5b5486 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -139,36 +139,39 @@ public final class ConfigConstants {
 	public static final String RESOURCE_MANAGER_IPC_PORT_KEY = "resourcemanager.rpc.port";
 
 	/**
-	 * The config parameter defining the storage directory to be used by the blob server.
+	 * @deprecated use {@link BlobServerOptions#STORAGE_DIRECTORY} instead
 	 */
+	@Deprecated
 	public static final String BLOB_STORAGE_DIRECTORY_KEY = "blob.storage.directory";
 
 	/**
-	 * The config parameter defining number of retires for failed BLOB fetches.
+	 * @deprecated use {@link BlobServerOptions#FETCH_RETRIES} instead
 	 */
+	@Deprecated
 	public static final String BLOB_FETCH_RETRIES_KEY = "blob.fetch.retries";
 
 	/**
-	 * The config parameter defining the maximum number of concurrent BLOB fetches that the JobManager serves.
+	 * @deprecated use {@link BlobServerOptions#FETCH_CONCURRENT} instead
 	 */
+	@Deprecated
 	public static final String BLOB_FETCH_CONCURRENT_KEY = "blob.fetch.num-concurrent";
 
 	/**
-	 * The config parameter defining the backlog of BLOB fetches on the JobManager
+	 * @deprecated use {@link BlobServerOptions#FETCH_BACKLOG} instead
 	 */
+	@Deprecated
 	public static final String BLOB_FETCH_BACKLOG_KEY = "blob.fetch.backlog";
 
 	/**
-	 * The config parameter defining the server port of the blob service.
-	 * The port can either be a port, such as "9123",
-	 * a range of ports: "50100-50200"
-	 * or a list of ranges and or points: "50100-50200,50300-50400,51234"
-	 *
-	 * Setting the port to 0 will let the OS choose an available port.
+	 * @deprecated use {@link BlobServerOptions#PORT} instead
 	 */
+	@Deprecated
 	public static final String BLOB_SERVER_PORT = "blob.server.port";
 
-	/** Flag to override ssl support for the blob service transport */
+	/**
+	 * @deprecated use {@link BlobServerOptions#SSL_ENABLED} instead
+	 */
+	@Deprecated
 	public static final String BLOB_SERVICE_SSL_ENABLED = "blob.service.ssl.enabled";
 
 	/**
@@ -1094,28 +1097,33 @@ public final class ConfigConstants {
 	public static final int DEFAULT_RESOURCE_MANAGER_IPC_PORT = 0;
 
 	/**
-	 * The default value to override ssl support for blob service transport
+	 * @deprecated use {@link BlobServerOptions#SSL_ENABLED} instead
 	 */
+	@Deprecated
 	public static final boolean DEFAULT_BLOB_SERVICE_SSL_ENABLED = true;
 
 	/**
-	 * Default number of retries for failed BLOB fetches.
+	 * @deprecated use {@link BlobServerOptions#FETCH_RETRIES} instead
 	 */
+	@Deprecated
 	public static final int DEFAULT_BLOB_FETCH_RETRIES = 5;
 
 	/**
-	 * Default number of concurrent BLOB fetch operations.
+	 * @deprecated use {@link BlobServerOptions#FETCH_CONCURRENT} instead
 	 */
+	@Deprecated
 	public static final int DEFAULT_BLOB_FETCH_CONCURRENT = 50;
 
 	/**
-	 * Default BLOB fetch connection backlog.
+	 * @deprecated use {@link BlobServerOptions#FETCH_BACKLOG} instead
 	 */
+	@Deprecated
 	public static final int DEFAULT_BLOB_FETCH_BACKLOG = 1000;
 
 	/**
-	 * Default BLOB server port. 0 means ephemeral port.
+	 * @deprecated use {@link BlobServerOptions#PORT} instead
 	 */
+	@Deprecated
 	public static final String DEFAULT_BLOB_SERVER_PORT = "0";
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
index 2587b15..23c7e63 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.blob;
 
-import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.util.FileUtils;
@@ -129,19 +129,18 @@ public final class BlobCache implements BlobService {
 		this.blobStore = blobStore;
 
 		// configure and create the storage directory
-		String storageDirectory = blobClientConfig.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null);
+		String storageDirectory = blobClientConfig.getString(BlobServerOptions.STORAGE_DIRECTORY);
 		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
 		LOG.info("Created BLOB cache storage directory " + storageDir);
 
 		// configure the number of fetch retries
-		final int fetchRetries = blobClientConfig.getInteger(
-			ConfigConstants.BLOB_FETCH_RETRIES_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_RETRIES);
+		final int fetchRetries = blobClientConfig.getInteger(BlobServerOptions.FETCH_RETRIES);
 		if (fetchRetries >= 0) {
 			this.numFetchRetries = fetchRetries;
 		}
 		else {
 			LOG.warn("Invalid value for {}. System will attempt no retires on failed fetches of BLOBs.",
-				ConfigConstants.BLOB_FETCH_RETRIES_KEY);
+				BlobServerOptions.FETCH_RETRIES.key());
 			this.numFetchRetries = 0;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
index ea90f54..49e54a1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.blob;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.fs.FileSystem;
@@ -92,8 +92,7 @@ public final class BlobClient implements Closeable {
 			// Check if ssl is enabled
 			SSLContext clientSSLContext = null;
 			if (clientConfig != null &&
-				clientConfig.getBoolean(ConfigConstants.BLOB_SERVICE_SSL_ENABLED,
-						ConfigConstants.DEFAULT_BLOB_SERVICE_SSL_ENABLED)) {
+				clientConfig.getBoolean(BlobServerOptions.SSL_ENABLED)) {
 
 				clientSSLContext = SSLUtils.createSSLClientContext(clientConfig);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index 8a70559..0e15777 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.blob;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
@@ -111,34 +111,32 @@ public class BlobServer extends Thread implements BlobService {
 		this.blobStore = checkNotNull(blobStore);
 
 		// configure and create the storage directory
-		String storageDirectory = config.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null);
+		String storageDirectory = config.getString(BlobServerOptions.STORAGE_DIRECTORY);
 		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
 		LOG.info("Created BLOB server storage directory {}", storageDir);
 
 		// configure the maximum number of concurrent connections
-		final int maxConnections = config.getInteger(
-				ConfigConstants.BLOB_FETCH_CONCURRENT_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT);
+		final int maxConnections = config.getInteger(BlobServerOptions.FETCH_CONCURRENT);
 		if (maxConnections >= 1) {
 			this.maxConnections = maxConnections;
 		}
 		else {
 			LOG.warn("Invalid value for maximum connections in BLOB server: {}. Using default value of {}",
-					maxConnections, ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT);
-			this.maxConnections = ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT;
+					maxConnections, BlobServerOptions.FETCH_CONCURRENT.defaultValue());
+			this.maxConnections = BlobServerOptions.FETCH_CONCURRENT.defaultValue();
 		}
 
 		// configure the backlog of connections
-		int backlog = config.getInteger(ConfigConstants.BLOB_FETCH_BACKLOG_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG);
+		int backlog = config.getInteger(BlobServerOptions.FETCH_BACKLOG);
 		if (backlog < 1) {
 			LOG.warn("Invalid value for BLOB connection backlog: {}. Using default value of {}",
-					backlog, ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG);
-			backlog = ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG;
+					backlog, BlobServerOptions.FETCH_BACKLOG.defaultValue());
+			backlog = BlobServerOptions.FETCH_BACKLOG.defaultValue();
 		}
 
 		this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
 
-		if (config.getBoolean(ConfigConstants.BLOB_SERVICE_SSL_ENABLED,
-				ConfigConstants.DEFAULT_BLOB_SERVICE_SSL_ENABLED)) {
+		if (config.getBoolean(BlobServerOptions.SSL_ENABLED)) {
 			try {
 				serverSSLContext = SSLUtils.createSSLServerContext(config);
 			} catch (Exception e) {
@@ -148,7 +146,7 @@ public class BlobServer extends Thread implements BlobService {
 
 		//  ----------------------- start the server -------------------
 
-		String serverPortRange = config.getString(ConfigConstants.BLOB_SERVER_PORT, ConfigConstants.DEFAULT_BLOB_SERVER_PORT);
+		String serverPortRange = config.getString(BlobServerOptions.PORT);
 
 		Iterator<Integer> ports = NetUtils.getPortRangeFromString(serverPortRange);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientSslTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientSslTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientSslTest.java
index 5054107..27603d0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientSslTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientSslTest.java
@@ -31,6 +31,7 @@ import java.security.MessageDigest;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.api.common.JobID;
@@ -91,7 +92,7 @@ public class BlobClientSslTest {
 		try {
 			Configuration config = new Configuration();
 			config.setBoolean(ConfigConstants.SECURITY_SSL_ENABLED, true);
-			config.setBoolean(ConfigConstants.BLOB_SERVICE_SSL_ENABLED, false);
+			config.setBoolean(BlobServerOptions.SSL_ENABLED, false);
 			config.setString(ConfigConstants.SECURITY_SSL_KEYSTORE, "src/test/resources/local127.keystore");
 			config.setString(ConfigConstants.SECURITY_SSL_KEYSTORE_PASSWORD, "password");
 			config.setString(ConfigConstants.SECURITY_SSL_KEY_PASSWORD, "password");
@@ -104,7 +105,7 @@ public class BlobClientSslTest {
 
 		clientConfig = new Configuration();
 		clientConfig.setBoolean(ConfigConstants.SECURITY_SSL_ENABLED, true);
-		clientConfig.setBoolean(ConfigConstants.BLOB_SERVICE_SSL_ENABLED, false);
+		clientConfig.setBoolean(BlobServerOptions.SSL_ENABLED, false);
 		clientConfig.setString(ConfigConstants.SECURITY_SSL_TRUSTSTORE, "src/test/resources/local127.truststore");
 		clientConfig.setString(ConfigConstants.SECURITY_SSL_TRUSTSTORE_PASSWORD, "password");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
index ea0eb94..c3762aa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRangeTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.blob;
 
-import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.NetUtils;
 import org.apache.flink.util.TestLogger;
@@ -38,7 +38,7 @@ public class BlobServerRangeTest extends TestLogger {
 	@Test
 	public void testOnEphemeralPort() throws IOException {
 		Configuration conf = new Configuration();
-		conf.setString(ConfigConstants.BLOB_SERVER_PORT, "0");
+		conf.setString(BlobServerOptions.PORT, "0");
 		BlobServer srv = new BlobServer(conf);
 		srv.shutdown();
 	}
@@ -59,7 +59,7 @@ public class BlobServerRangeTest extends TestLogger {
 		}
 
 		Configuration conf = new Configuration();
-		conf.setString(ConfigConstants.BLOB_SERVER_PORT, String.valueOf(socket.getLocalPort()));
+		conf.setString(BlobServerOptions.PORT, String.valueOf(socket.getLocalPort()));
 
 		// this thing is going to throw an exception
 		try {
@@ -88,7 +88,7 @@ public class BlobServerRangeTest extends TestLogger {
 		}
 		int availablePort = NetUtils.getAvailablePort();
 		Configuration conf = new Configuration();
-		conf.setString(ConfigConstants.BLOB_SERVER_PORT, sockets[0].getLocalPort() + "," + sockets[1].getLocalPort() + "," + availablePort);
+		conf.setString(BlobServerOptions.PORT, sockets[0].getLocalPort() + "," + sockets[1].getLocalPort() + "," + availablePort);
 
 		// this thing is going to throw an exception
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/5e61a01b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
index 9ac6873..a906d9b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
@@ -30,7 +30,7 @@ import java.util.List;
 
 import com.google.common.io.Files;
 
-import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.util.StartupUtils;
 import org.apache.flink.util.NetUtils;
@@ -130,7 +130,7 @@ public class JobManagerStartupTest extends TestLogger {
 		}
 		Configuration failConfig = new Configuration();
 		String nonExistDirectory = new File(blobStorageDirectory, DOES_NOT_EXISTS_NO_SIR).getAbsolutePath();
-		failConfig.setString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, nonExistDirectory);
+		failConfig.setString(BlobServerOptions.STORAGE_DIRECTORY, nonExistDirectory);
 
 		try {
 			JobManager.runJobManager(failConfig, JobManagerMode.CLUSTER, "localhost", portNum);


[03/14] flink git commit: Update Flink version to 1.4-SNAPSHOT

Posted by sr...@apache.org.
Update Flink version to 1.4-SNAPSHOT


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

Branch: refs/heads/release-1.3
Commit: fa49915afd6d5755636047a343dd92c092320f16
Parents: 670b23e
Author: Robert Metzger <rm...@apache.org>
Authored: Mon May 8 09:17:43 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:06:28 2017 +0200

----------------------------------------------------------------------
 docs/_config.yml                                                | 5 +++--
 flink-annotations/pom.xml                                       | 2 +-
 flink-clients/pom.xml                                           | 2 +-
 flink-connectors/flink-avro/pom.xml                             | 2 +-
 flink-connectors/flink-connector-cassandra/pom.xml              | 2 +-
 flink-connectors/flink-connector-elasticsearch-base/pom.xml     | 2 +-
 flink-connectors/flink-connector-elasticsearch/pom.xml          | 2 +-
 flink-connectors/flink-connector-elasticsearch2/pom.xml         | 2 +-
 flink-connectors/flink-connector-elasticsearch5/pom.xml         | 2 +-
 flink-connectors/flink-connector-filesystem/pom.xml             | 2 +-
 flink-connectors/flink-connector-kafka-0.10/pom.xml             | 2 +-
 flink-connectors/flink-connector-kafka-0.8/pom.xml              | 2 +-
 flink-connectors/flink-connector-kafka-0.9/pom.xml              | 2 +-
 flink-connectors/flink-connector-kafka-base/pom.xml             | 2 +-
 flink-connectors/flink-connector-kinesis/pom.xml                | 2 +-
 flink-connectors/flink-connector-nifi/pom.xml                   | 2 +-
 flink-connectors/flink-connector-rabbitmq/pom.xml               | 2 +-
 flink-connectors/flink-connector-twitter/pom.xml                | 2 +-
 flink-connectors/flink-hadoop-compatibility/pom.xml             | 2 +-
 flink-connectors/flink-hbase/pom.xml                            | 2 +-
 flink-connectors/flink-hcatalog/pom.xml                         | 2 +-
 flink-connectors/flink-jdbc/pom.xml                             | 2 +-
 flink-connectors/pom.xml                                        | 2 +-
 flink-contrib/flink-connector-wikiedits/pom.xml                 | 2 +-
 flink-contrib/flink-statebackend-rocksdb/pom.xml                | 2 +-
 flink-contrib/flink-storm-examples/pom.xml                      | 2 +-
 flink-contrib/flink-storm/pom.xml                               | 2 +-
 flink-contrib/flink-streaming-contrib/pom.xml                   | 2 +-
 flink-contrib/flink-tweet-inputformat/pom.xml                   | 2 +-
 flink-contrib/pom.xml                                           | 2 +-
 flink-core/pom.xml                                              | 2 +-
 flink-dist/pom.xml                                              | 2 +-
 flink-examples/flink-examples-batch/pom.xml                     | 2 +-
 flink-examples/flink-examples-streaming/pom.xml                 | 2 +-
 flink-examples/flink-examples-table/pom.xml                     | 2 +-
 flink-examples/pom.xml                                          | 2 +-
 flink-fs-tests/pom.xml                                          | 2 +-
 flink-java/pom.xml                                              | 2 +-
 flink-java8/pom.xml                                             | 2 +-
 flink-libraries/flink-cep-scala/pom.xml                         | 2 +-
 flink-libraries/flink-cep/pom.xml                               | 2 +-
 flink-libraries/flink-gelly-examples/pom.xml                    | 2 +-
 flink-libraries/flink-gelly-scala/pom.xml                       | 2 +-
 flink-libraries/flink-gelly/pom.xml                             | 2 +-
 flink-libraries/flink-ml/pom.xml                                | 2 +-
 flink-libraries/flink-python/pom.xml                            | 2 +-
 flink-libraries/flink-table/pom.xml                             | 2 +-
 flink-libraries/pom.xml                                         | 2 +-
 flink-mesos/pom.xml                                             | 2 +-
 flink-metrics/flink-metrics-core/pom.xml                        | 2 +-
 flink-metrics/flink-metrics-dropwizard/pom.xml                  | 2 +-
 flink-metrics/flink-metrics-ganglia/pom.xml                     | 2 +-
 flink-metrics/flink-metrics-graphite/pom.xml                    | 2 +-
 flink-metrics/flink-metrics-jmx/pom.xml                         | 2 +-
 flink-metrics/flink-metrics-statsd/pom.xml                      | 2 +-
 flink-metrics/pom.xml                                           | 2 +-
 flink-optimizer/pom.xml                                         | 2 +-
 flink-quickstart/flink-quickstart-java/pom.xml                  | 2 +-
 .../src/main/resources/archetype-resources/pom.xml              | 2 +-
 flink-quickstart/flink-quickstart-scala/pom.xml                 | 2 +-
 .../src/main/resources/archetype-resources/pom.xml              | 2 +-
 flink-quickstart/pom.xml                                        | 2 +-
 flink-runtime-web/pom.xml                                       | 2 +-
 flink-runtime/pom.xml                                           | 2 +-
 flink-scala-shell/pom.xml                                       | 2 +-
 flink-scala/pom.xml                                             | 2 +-
 flink-shaded-curator/flink-shaded-curator-recipes/pom.xml       | 2 +-
 flink-shaded-curator/flink-shaded-curator-test/pom.xml          | 2 +-
 flink-shaded-curator/pom.xml                                    | 2 +-
 flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml                | 2 +-
 flink-shaded-hadoop/flink-shaded-include-yarn-tests/pom.xml     | 2 +-
 flink-shaded-hadoop/pom.xml                                     | 2 +-
 flink-streaming-java/pom.xml                                    | 2 +-
 flink-streaming-scala/pom.xml                                   | 2 +-
 flink-test-utils-parent/flink-test-utils-junit/pom.xml          | 2 +-
 flink-test-utils-parent/flink-test-utils/pom.xml                | 2 +-
 flink-test-utils-parent/pom.xml                                 | 2 +-
 flink-tests/pom.xml                                             | 2 +-
 flink-yarn-tests/pom.xml                                        | 2 +-
 flink-yarn/pom.xml                                              | 2 +-
 pom.xml                                                         | 4 ++--
 tools/change-version.sh                                         | 4 ++--
 tools/force-shading/pom.xml                                     | 2 +-
 83 files changed, 87 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 9167ace..cf5ee10 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -26,8 +26,8 @@
 # to reference a specific Flink version, because this is the only place where
 # we change the version for the complete docs when forking of a release branch
 # etc.
-version: "1.3-SNAPSHOT"
-version_short: "1.3" # Used for the top navbar w/o snapshot suffix
+version: "1.4-SNAPSHOT"
+version_short: "1.4" # Used for the top navbar w/o snapshot suffix
 is_snapshot_version: true
 
 # This suffix is appended to the Scala-dependent Maven artifact names
@@ -46,6 +46,7 @@ is_latest: true
 is_stable: false
 
 previous_docs:
+  1.3: http://ci.apache.org/projects/flink/flink-docs-release-1.3
   1.2: http://ci.apache.org/projects/flink/flink-docs-release-1.2
   1.1: http://ci.apache.org/projects/flink/flink-docs-release-1.1
   1.0: http://ci.apache.org/projects/flink/flink-docs-release-1.0

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-annotations/pom.xml
----------------------------------------------------------------------
diff --git a/flink-annotations/pom.xml b/flink-annotations/pom.xml
index f3013a2..b375611 100644
--- a/flink-annotations/pom.xml
+++ b/flink-annotations/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-clients/pom.xml
----------------------------------------------------------------------
diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml
index 0e0c146..8dde2e9 100644
--- a/flink-clients/pom.xml
+++ b/flink-clients/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/pom.xml b/flink-connectors/flink-avro/pom.xml
index 76914ee..170b344 100644
--- a/flink-connectors/flink-avro/pom.xml
+++ b/flink-connectors/flink-avro/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml
index 3080938..d01c769 100644
--- a/flink-connectors/flink-connector-cassandra/pom.xml
+++ b/flink-connectors/flink-connector-cassandra/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-elasticsearch-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
index 32327ff..e183354 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch/pom.xml b/flink-connectors/flink-connector-elasticsearch/pom.xml
index eebdb4b..07028df 100644
--- a/flink-connectors/flink-connector-elasticsearch/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-elasticsearch2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml
index 30396dd..ec1b7da 100644
--- a/flink-connectors/flink-connector-elasticsearch2/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-elasticsearch5/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml
index a0bd328..93d7bbe 100644
--- a/flink-connectors/flink-connector-elasticsearch5/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml
index a37cb68..28d9962 100644
--- a/flink-connectors/flink-connector-filesystem/pom.xml
+++ b/flink-connectors/flink-connector-filesystem/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
index 9fb0e61..c17386d 100644
--- a/flink-connectors/flink-connector-kafka-0.10/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml
index ff35300..ceccb3e 100644
--- a/flink-connectors/flink-connector-kafka-0.8/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
index aa79a1d..de4dedb 100644
--- a/flink-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml
index 263eb9a..a6b2349 100644
--- a/flink-connectors/flink-connector-kafka-base/pom.xml
+++ b/flink-connectors/flink-connector-kafka-base/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-kinesis/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml
index 080626f..7515f31 100644
--- a/flink-connectors/flink-connector-kinesis/pom.xml
+++ b/flink-connectors/flink-connector-kinesis/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-nifi/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-nifi/pom.xml b/flink-connectors/flink-connector-nifi/pom.xml
index 7e9949f..67bfb29 100644
--- a/flink-connectors/flink-connector-nifi/pom.xml
+++ b/flink-connectors/flink-connector-nifi/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-rabbitmq/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/pom.xml b/flink-connectors/flink-connector-rabbitmq/pom.xml
index 6973a0e..f26e57c 100644
--- a/flink-connectors/flink-connector-rabbitmq/pom.xml
+++ b/flink-connectors/flink-connector-rabbitmq/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-connector-twitter/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-twitter/pom.xml b/flink-connectors/flink-connector-twitter/pom.xml
index 3e68541..c683da1 100644
--- a/flink-connectors/flink-connector-twitter/pom.xml
+++ b/flink-connectors/flink-connector-twitter/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-hadoop-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml
index 6f20089..003225b 100644
--- a/flink-connectors/flink-hadoop-compatibility/pom.xml
+++ b/flink-connectors/flink-hadoop-compatibility/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml
index 4fc0556..6a295a8 100644
--- a/flink-connectors/flink-hbase/pom.xml
+++ b/flink-connectors/flink-hbase/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml
index ba0e142..a975e53 100644
--- a/flink-connectors/flink-hcatalog/pom.xml
+++ b/flink-connectors/flink-hcatalog/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/flink-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-jdbc/pom.xml b/flink-connectors/flink-jdbc/pom.xml
index e96ab69..38a0163 100644
--- a/flink-connectors/flink-jdbc/pom.xml
+++ b/flink-connectors/flink-jdbc/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-connectors</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index 5d8ca70..5af85b8 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/flink-connector-wikiedits/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-connector-wikiedits/pom.xml b/flink-contrib/flink-connector-wikiedits/pom.xml
index 976e0dc..fed8656 100644
--- a/flink-contrib/flink-connector-wikiedits/pom.xml
+++ b/flink-contrib/flink-connector-wikiedits/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-contrib</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/flink-statebackend-rocksdb/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/pom.xml b/flink-contrib/flink-statebackend-rocksdb/pom.xml
index bd890d9..2148706 100644
--- a/flink-contrib/flink-statebackend-rocksdb/pom.xml
+++ b/flink-contrib/flink-statebackend-rocksdb/pom.xml
@@ -27,7 +27,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-contrib</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/flink-storm-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml
index d042dde..4d9d76c 100644
--- a/flink-contrib/flink-storm-examples/pom.xml
+++ b/flink-contrib/flink-storm-examples/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-contrib</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/flink-storm/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml
index cc300b3..5189b3b 100644
--- a/flink-contrib/flink-storm/pom.xml
+++ b/flink-contrib/flink-storm/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-contrib</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/flink-streaming-contrib/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-streaming-contrib/pom.xml b/flink-contrib/flink-streaming-contrib/pom.xml
index a218f12..2df0231 100644
--- a/flink-contrib/flink-streaming-contrib/pom.xml
+++ b/flink-contrib/flink-streaming-contrib/pom.xml
@@ -27,7 +27,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-contrib</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/flink-tweet-inputformat/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/pom.xml b/flink-contrib/flink-tweet-inputformat/pom.xml
index 95b6bd8..5e0d79c 100644
--- a/flink-contrib/flink-tweet-inputformat/pom.xml
+++ b/flink-contrib/flink-tweet-inputformat/pom.xml
@@ -27,7 +27,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-contrib</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-contrib/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml
index 75770a4..2277465 100644
--- a/flink-contrib/pom.xml
+++ b/flink-contrib/pom.xml
@@ -27,7 +27,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index 1efcb29..6b0d778 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 4b8b2c3..6d8debf 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-examples/flink-examples-batch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-batch/pom.xml b/flink-examples/flink-examples-batch/pom.xml
index 20dc951..269f5a6 100644
--- a/flink-examples/flink-examples-batch/pom.xml
+++ b/flink-examples/flink-examples-batch/pom.xml
@@ -24,7 +24,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-examples_2.10</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-examples/flink-examples-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index adc819e..2bacf25 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-examples_2.10</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-examples/flink-examples-table/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml
index 684d883..ce1c701 100644
--- a/flink-examples/flink-examples-table/pom.xml
+++ b/flink-examples/flink-examples-table/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-examples_2.10</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml
index a5939f5..409c8f1 100644
--- a/flink-examples/pom.xml
+++ b/flink-examples/pom.xml
@@ -24,7 +24,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-fs-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml
index 84bba0a..c354c80 100644
--- a/flink-fs-tests/pom.xml
+++ b/flink-fs-tests/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java/pom.xml b/flink-java/pom.xml
index 55b3975..24fc0f6 100644
--- a/flink-java/pom.xml
+++ b/flink-java/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-java8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml
index fbc5b3a..8de3b92 100644
--- a/flink-java8/pom.xml
+++ b/flink-java8/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-cep-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/pom.xml b/flink-libraries/flink-cep-scala/pom.xml
index 57c594e..ca5bbce 100644
--- a/flink-libraries/flink-cep-scala/pom.xml
+++ b/flink-libraries/flink-cep-scala/pom.xml
@@ -26,7 +26,7 @@ under the License.
     <parent>
         <groupId>org.apache.flink</groupId>
         <artifactId>flink-libraries</artifactId>
-        <version>1.3-SNAPSHOT</version>
+        <version>1.4-SNAPSHOT</version>
         <relativePath>..</relativePath>
     </parent>
     

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-cep/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml
index 1c9f8be..7f89c24 100644
--- a/flink-libraries/flink-cep/pom.xml
+++ b/flink-libraries/flink-cep/pom.xml
@@ -26,7 +26,7 @@ under the License.
     <parent>
         <groupId>org.apache.flink</groupId>
         <artifactId>flink-libraries</artifactId>
-        <version>1.3-SNAPSHOT</version>
+        <version>1.4-SNAPSHOT</version>
         <relativePath>..</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-gelly-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/pom.xml b/flink-libraries/flink-gelly-examples/pom.xml
index cd8d807..009781f 100644
--- a/flink-libraries/flink-gelly-examples/pom.xml
+++ b/flink-libraries/flink-gelly-examples/pom.xml
@@ -23,7 +23,7 @@
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-libraries</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-gelly-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/pom.xml b/flink-libraries/flink-gelly-scala/pom.xml
index dad7237..256fc53 100644
--- a/flink-libraries/flink-gelly-scala/pom.xml
+++ b/flink-libraries/flink-gelly-scala/pom.xml
@@ -23,7 +23,7 @@ under the License.
     <parent>
         <groupId>org.apache.flink</groupId>
         <artifactId>flink-libraries</artifactId>
-        <version>1.3-SNAPSHOT</version>
+        <version>1.4-SNAPSHOT</version>
         <relativePath>..</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-gelly/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml
index fa09102..d656bb3 100644
--- a/flink-libraries/flink-gelly/pom.xml
+++ b/flink-libraries/flink-gelly/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-libraries</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-ml/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/pom.xml b/flink-libraries/flink-ml/pom.xml
index bea956e..8aa84d2 100644
--- a/flink-libraries/flink-ml/pom.xml
+++ b/flink-libraries/flink-ml/pom.xml
@@ -24,7 +24,7 @@
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-libraries</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-python/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-python/pom.xml b/flink-libraries/flink-python/pom.xml
index 8f75f25..9d2f0c3 100644
--- a/flink-libraries/flink-python/pom.xml
+++ b/flink-libraries/flink-python/pom.xml
@@ -23,7 +23,7 @@ under the License.
     <parent>
         <groupId>org.apache.flink</groupId>
         <artifactId>flink-libraries</artifactId>
-        <version>1.3-SNAPSHOT</version>
+        <version>1.4-SNAPSHOT</version>
         <relativePath>..</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/flink-table/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml
index c7440ab..8fa2ed2 100644
--- a/flink-libraries/flink-table/pom.xml
+++ b/flink-libraries/flink-table/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-libraries</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-libraries/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/pom.xml b/flink-libraries/pom.xml
index 6dbc692..5b4cf79 100644
--- a/flink-libraries/pom.xml
+++ b/flink-libraries/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-mesos/pom.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml
index eea5da7..d03144f 100644
--- a/flink-mesos/pom.xml
+++ b/flink-mesos/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/flink-metrics-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-core/pom.xml b/flink-metrics/flink-metrics-core/pom.xml
index 658847f..5f3a7a50 100644
--- a/flink-metrics/flink-metrics-core/pom.xml
+++ b/flink-metrics/flink-metrics-core/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-metrics</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/flink-metrics-dropwizard/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml
index de57729..2ba12e6 100644
--- a/flink-metrics/flink-metrics-dropwizard/pom.xml
+++ b/flink-metrics/flink-metrics-dropwizard/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-metrics</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/flink-metrics-ganglia/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-ganglia/pom.xml b/flink-metrics/flink-metrics-ganglia/pom.xml
index 8f39edf..244dee7 100644
--- a/flink-metrics/flink-metrics-ganglia/pom.xml
+++ b/flink-metrics/flink-metrics-ganglia/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-metrics</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/flink-metrics-graphite/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml
index f8a21a0..b7d02bd 100644
--- a/flink-metrics/flink-metrics-graphite/pom.xml
+++ b/flink-metrics/flink-metrics-graphite/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-metrics</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/flink-metrics-jmx/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-jmx/pom.xml b/flink-metrics/flink-metrics-jmx/pom.xml
index 8063deb..d738fc4 100644
--- a/flink-metrics/flink-metrics-jmx/pom.xml
+++ b/flink-metrics/flink-metrics-jmx/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-metrics</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/flink-metrics-statsd/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml
index a243b92..80cfa32 100644
--- a/flink-metrics/flink-metrics-statsd/pom.xml
+++ b/flink-metrics/flink-metrics-statsd/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-metrics</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml
index 2ecf085..e1b66c2 100644
--- a/flink-metrics/pom.xml
+++ b/flink-metrics/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-optimizer/pom.xml
----------------------------------------------------------------------
diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml
index 11e2e16..c347082 100644
--- a/flink-optimizer/pom.xml
+++ b/flink-optimizer/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-quickstart/flink-quickstart-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/pom.xml b/flink-quickstart/flink-quickstart-java/pom.xml
index 31b45f7..94574dc 100644
--- a/flink-quickstart/flink-quickstart-java/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/pom.xml
@@ -27,7 +27,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-quickstart</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index 1e525fd..2fa1835 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -30,7 +30,7 @@ under the License.
 
 	<properties>
 		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<flink.version>1.3-SNAPSHOT</flink.version>
+		<flink.version>1.4-SNAPSHOT</flink.version>
 		<slf4j.version>1.7.7</slf4j.version>
 		<log4j.version>1.2.17</log4j.version>
 	</properties>

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-quickstart/flink-quickstart-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/pom.xml b/flink-quickstart/flink-quickstart-scala/pom.xml
index 4a5bc97..875fcca 100644
--- a/flink-quickstart/flink-quickstart-scala/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/pom.xml
@@ -27,7 +27,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-quickstart</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index 5abe496..189bbce 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -45,7 +45,7 @@ under the License.
 
 	<properties>
 		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<flink.version>1.3-SNAPSHOT</flink.version>
+		<flink.version>1.4-SNAPSHOT</flink.version>
 		<slf4j.version>1.7.7</slf4j.version>
 		<log4j.version>1.2.17</log4j.version>
 	</properties>

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-quickstart/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/pom.xml b/flink-quickstart/pom.xml
index 068bbc9..7e1b460 100644
--- a/flink-quickstart/pom.xml
+++ b/flink-quickstart/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-runtime-web/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml
index a50e01f..e7870f3 100644
--- a/flink-runtime-web/pom.xml
+++ b/flink-runtime-web/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 2b42cc5..8abdc3c 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-scala-shell/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala-shell/pom.xml b/flink-scala-shell/pom.xml
index cf637d4..0aadfc6 100644
--- a/flink-scala-shell/pom.xml
+++ b/flink-scala-shell/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index 6c9a933..7be1e29 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -24,7 +24,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
index 5295a1a..fcc2759 100644
--- a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
+++ b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-shaded-curator</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-shaded-curator/flink-shaded-curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/flink-shaded-curator-test/pom.xml b/flink-shaded-curator/flink-shaded-curator-test/pom.xml
index fd9f1ce..751b590 100644
--- a/flink-shaded-curator/flink-shaded-curator-test/pom.xml
+++ b/flink-shaded-curator/flink-shaded-curator-test/pom.xml
@@ -24,7 +24,7 @@
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-shaded-curator</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-shaded-curator/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/pom.xml b/flink-shaded-curator/pom.xml
index be1da9c..d08320d 100644
--- a/flink-shaded-curator/pom.xml
+++ b/flink-shaded-curator/pom.xml
@@ -24,7 +24,7 @@
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
index c750bbd..9ac033e 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-shaded-hadoop</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-shaded-hadoop/flink-shaded-include-yarn-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-include-yarn-tests/pom.xml b/flink-shaded-hadoop/flink-shaded-include-yarn-tests/pom.xml
index b7b2229..fe87331 100644
--- a/flink-shaded-hadoop/flink-shaded-include-yarn-tests/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-include-yarn-tests/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-shaded-hadoop</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-shaded-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml
index 3df0756..1d957ab 100644
--- a/flink-shaded-hadoop/pom.xml
+++ b/flink-shaded-hadoop/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-streaming-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml
index 013e958..ea987af 100644
--- a/flink-streaming-java/pom.xml
+++ b/flink-streaming-java/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-streaming-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml
index f95ad79..4ec12bd 100644
--- a/flink-streaming-scala/pom.xml
+++ b/flink-streaming-scala/pom.xml
@@ -24,7 +24,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-test-utils-parent/flink-test-utils-junit/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils-junit/pom.xml b/flink-test-utils-parent/flink-test-utils-junit/pom.xml
index aefd62d..c8469cd 100644
--- a/flink-test-utils-parent/flink-test-utils-junit/pom.xml
+++ b/flink-test-utils-parent/flink-test-utils-junit/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-test-utils-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-test-utils-parent/flink-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml
index 14fca07..36bdceb 100644
--- a/flink-test-utils-parent/flink-test-utils/pom.xml
+++ b/flink-test-utils-parent/flink-test-utils/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-test-utils-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-test-utils-parent/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/pom.xml b/flink-test-utils-parent/pom.xml
index 92d6be3..48e4550 100644
--- a/flink-test-utils-parent/pom.xml
+++ b/flink-test-utils-parent/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 62feff3..b67edbb 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-yarn-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index 4b97596..f296e8d 100644
--- a/flink-yarn-tests/pom.xml
+++ b/flink-yarn-tests/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/flink-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml
index 11d238a..5946094 100644
--- a/flink-yarn/pom.xml
+++ b/flink-yarn/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.3-SNAPSHOT</version>
+		<version>1.4-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f0bf7a5..db19b9d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -28,7 +28,7 @@ under the License.
 
 	<groupId>org.apache.flink</groupId>
 	<artifactId>flink-parent</artifactId>
-	<version>1.3-SNAPSHOT</version>
+	<version>1.4-SNAPSHOT</version>
 
 	<name>flink</name>
 	<packaging>pom</packaging>
@@ -121,7 +121,7 @@ under the License.
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>force-shading</artifactId>
-			<version>1.3-SNAPSHOT</version>
+			<version>1.4-SNAPSHOT</version>
 		</dependency>
 
 		<!-- Root dependencies for all projects -->

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/tools/change-version.sh
----------------------------------------------------------------------
diff --git a/tools/change-version.sh b/tools/change-version.sh
index 45ed2fa..cafefd0 100755
--- a/tools/change-version.sh
+++ b/tools/change-version.sh
@@ -17,8 +17,8 @@
 # limitations under the License.
 ################################################################################
 
-OLD="1.2-SNAPSHOT"
-NEW="1.3-SNAPSHOT"
+OLD="1.3-SNAPSHOT"
+NEW="1.4-SNAPSHOT"
 
 
 HERE=` basename "$PWD"`

http://git-wip-us.apache.org/repos/asf/flink/blob/fa49915a/tools/force-shading/pom.xml
----------------------------------------------------------------------
diff --git a/tools/force-shading/pom.xml b/tools/force-shading/pom.xml
index 3de6a5a..bbad337 100644
--- a/tools/force-shading/pom.xml
+++ b/tools/force-shading/pom.xml
@@ -38,7 +38,7 @@ under the License.
 
 	<groupId>org.apache.flink</groupId>
 	<artifactId>force-shading</artifactId>
-	<version>1.3-SNAPSHOT</version>
+	<version>1.4-SNAPSHOT</version>
 
 	<packaging>jar</packaging>
 


[13/14] flink git commit: [FLINK-6534] [checkpoint] Use async IO to dispose state in SharedStateRegistry

Posted by sr...@apache.org.
[FLINK-6534] [checkpoint] Use async IO to dispose state in SharedStateRegistry


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

Branch: refs/heads/release-1.3
Commit: 471263cfe493dc1bbbd5a5733dbf918cc0872b9b
Parents: 7c6f348
Author: Stefan Richter <s....@data-artisans.com>
Authored: Fri May 12 16:01:05 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Sun May 14 14:07:26 2017 +0200

----------------------------------------------------------------------
 .../runtime/checkpoint/AbstractCompletedCheckpointStore.java   | 6 ++++++
 .../runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java  | 2 ++
 .../org/apache/flink/runtime/state/SharedStateRegistry.java    | 6 +++++-
 3 files changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/471263cf/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCompletedCheckpointStore.java
index f42fd06..bf70501 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCompletedCheckpointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCompletedCheckpointStore.java
@@ -20,6 +20,8 @@ package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.runtime.state.SharedStateRegistry;
 
+import java.util.concurrent.Executor;
+
 /**
  * This is the base class that provides implementation of some aspects common for all
  * {@link CompletedCheckpointStore}s.
@@ -34,4 +36,8 @@ public abstract class AbstractCompletedCheckpointStore implements CompletedCheck
 	public AbstractCompletedCheckpointStore() {
 		this.sharedStateRegistry = new SharedStateRegistry();
 	}
+
+	public AbstractCompletedCheckpointStore(Executor asyncIOExecutor) {
+		this.sharedStateRegistry = new SharedStateRegistry(asyncIOExecutor);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/471263cf/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
index 52a4eea..c8c68bc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
@@ -106,6 +106,8 @@ public class ZooKeeperCompletedCheckpointStore extends AbstractCompletedCheckpoi
 			RetrievableStateStorageHelper<CompletedCheckpoint> stateStorage,
 			Executor executor) throws Exception {
 
+		super(executor);
+
 		checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint.");
 		checkNotNull(stateStorage, "State storage");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/471263cf/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
index 9cfdec7..f9161b0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
@@ -45,8 +45,12 @@ public class SharedStateRegistry {
 	private final Executor asyncDisposalExecutor;
 
 	public SharedStateRegistry() {
+		this(Executors.directExecutor());
+	}
+
+	public SharedStateRegistry(Executor asyncDisposalExecutor) {
 		this.registeredStates = new HashMap<>();
-		this.asyncDisposalExecutor = Executors.directExecutor(); //TODO: FLINK-6534
+		this.asyncDisposalExecutor = Preconditions.checkNotNull(asyncDisposalExecutor);
 	}
 
 	/**