You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by uc...@apache.org on 2016/04/29 10:05:52 UTC

[1/2] flink git commit: [FLINK-3790] [streaming] Use proper hadoop config in rolling sink [Forced Update!]

Repository: flink
Updated Branches:
  refs/heads/release-1.0.3-rc1 946c98d3e -> 06c5d7651 (forced update)


[FLINK-3790] [streaming] Use proper hadoop config in rolling sink

Closes #1919


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

Branch: refs/heads/release-1.0.3-rc1
Commit: f1d34b17b6171a6df6b18a0e417d39d02a4506b5
Parents: 81c69f9
Author: Gyula Fora <gy...@apache.org>
Authored: Wed Apr 20 22:22:55 2016 +0200
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Apr 28 21:54:41 2016 +0200

----------------------------------------------------------------------
 .../streaming/connectors/fs/RollingSink.java      | 18 ++++++++++--------
 .../connectors/fs/SequenceFileWriter.java         |  7 +++++--
 2 files changed, 15 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f1d34b17/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
index f186f53..9f235e8 100644
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
@@ -285,7 +286,9 @@ public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConf
 	 * current part file path, the valid length of the in-progress files and pending part files.
 	 */
 	private transient BucketState bucketState;
-
+	
+	private transient org.apache.hadoop.conf.Configuration hadoopConf;
+	
 	/**
 	 * Creates a new {@code RollingSink} that writes files to the given base directory.
 	 *
@@ -323,7 +326,8 @@ public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConf
 			bucketState = new BucketState();
 		}
 
-		FileSystem fs = new Path(basePath).getFileSystem(new org.apache.hadoop.conf.Configuration());
+		hadoopConf = HadoopFileSystem.getHadoopConfiguration();
+		FileSystem fs = new Path(basePath).getFileSystem(hadoopConf);
 		refTruncate = reflectTruncate(fs);
 
 		// delete pending/in-progress files that might be left if we fail while
@@ -418,9 +422,7 @@ public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConf
 	private void openNewPartFile() throws Exception {
 		closeCurrentPartFile();
 
-		org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration();
-
-		FileSystem fs = new Path(basePath).getFileSystem(conf);
+		FileSystem fs = new Path(basePath).getFileSystem(hadoopConf);
 
 		Path newBucketDirectory = bucketer.getNextBucketPath(new Path(basePath));
 
@@ -484,7 +486,7 @@ public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConf
 		if (currentPartPath != null) {
 			Path inProgressPath = new Path(currentPartPath.getParent(), inProgressPrefix + currentPartPath.getName()).suffix(inProgressSuffix);
 			Path pendingPath = new Path(currentPartPath.getParent(), pendingPrefix + currentPartPath.getName()).suffix(pendingSuffix);
-			FileSystem fs = inProgressPath.getFileSystem(new org.apache.hadoop.conf.Configuration());
+			FileSystem fs = inProgressPath.getFileSystem(hadoopConf);
 			fs.rename(inProgressPath, pendingPath);
 			LOG.debug("Moving in-progress bucket {} to pending file {}",
 					inProgressPath,
@@ -615,7 +617,7 @@ public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConf
 						Path pendingPath = new Path(finalPath.getParent(),
 								pendingPrefix + finalPath.getName()).suffix(pendingSuffix);
 
-						FileSystem fs = pendingPath.getFileSystem(new org.apache.hadoop.conf.Configuration());
+						FileSystem fs = pendingPath.getFileSystem(hadoopConf);
 						fs.rename(pendingPath, finalPath);
 						LOG.debug(
 								"Moving pending file {} to final location after complete checkpoint {}.",
@@ -657,7 +659,7 @@ public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConf
 		bucketState.pendingFiles.clear();
 		FileSystem fs = null;
 		try {
-			fs = new Path(basePath).getFileSystem(new org.apache.hadoop.conf.Configuration());
+			fs = new Path(basePath).getFileSystem(HadoopFileSystem.getHadoopConfiguration());
 		} catch (IOException e) {
 			LOG.error("Error while creating FileSystem in checkpoint restore.", e);
 			throw new RuntimeException("Error while creating FileSystem in checkpoint restore.", e);

http://git-wip-us.apache.org/repos/asf/flink/blob/f1d34b17/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
index 928d96e..3c8a58e 100644
--- a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.io.SequenceFile;
@@ -91,9 +92,11 @@ public class SequenceFileWriter<K extends Writable, V extends Writable> implemen
 		this.outputStream = outStream;
 
 		CompressionCodec codec = null;
+		
+		Configuration conf = HadoopFileSystem.getHadoopConfiguration();
 
 		if (!compressionCodecName.equals("None")) {
-			CompressionCodecFactory codecFactory = new CompressionCodecFactory(new Configuration());
+			CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf);
 			codec = codecFactory.getCodecByName(compressionCodecName);
 			if (codec == null) {
 				throw new RuntimeException("Codec " + compressionCodecName + " not found.");
@@ -101,7 +104,7 @@ public class SequenceFileWriter<K extends Writable, V extends Writable> implemen
 		}
 
 		// the non-deprecated constructor syntax is only available in recent hadoop versions...
-		writer = SequenceFile.createWriter(new Configuration(),
+		writer = SequenceFile.createWriter(conf,
 				outStream,
 				keyClass,
 				valueClass,


[2/2] flink git commit: Commit for release 1.0.3

Posted by uc...@apache.org.
Commit for release 1.0.3


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

Branch: refs/heads/release-1.0.3-rc1
Commit: 06c5d7651dd2ef670d94ba0d6f0bb922a0aafd1b
Parents: f1d34b1
Author: Ufuk Celebi <uc...@apache.org>
Authored: Thu Apr 28 20:28:50 2016 +0000
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Apr 28 20:28:50 2016 +0000

----------------------------------------------------------------------
 .gitignore                                      | 27 ----------
 .travis.yml                                     | 57 --------------------
 deploysettings.xml                              | 38 -------------
 docs/_config.yml                                |  4 +-
 flink-annotations/pom.xml                       |  2 +-
 flink-batch-connectors/flink-avro/pom.xml       |  2 +-
 .../flink-hadoop-compatibility/pom.xml          |  2 +-
 flink-batch-connectors/flink-hbase/pom.xml      |  2 +-
 flink-batch-connectors/flink-hcatalog/pom.xml   |  2 +-
 flink-batch-connectors/flink-jdbc/pom.xml       |  2 +-
 flink-batch-connectors/pom.xml                  |  2 +-
 flink-clients/pom.xml                           |  2 +-
 flink-contrib/flink-connector-wikiedits/pom.xml |  2 +-
 flink-contrib/flink-operator-stats/pom.xml      |  2 +-
 .../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/pom.xml                          |  2 +-
 flink-fs-tests/pom.xml                          |  2 +-
 flink-java/pom.xml                              |  2 +-
 flink-java8/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-optimizer/pom.xml                         |  2 +-
 flink-quickstart/flink-quickstart-java/pom.xml  |  2 +-
 .../main/resources/archetype-resources/pom.xml  |  2 +-
 flink-quickstart/flink-quickstart-scala/pom.xml |  2 +-
 .../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-recipes/pom.xml        |  2 +-
 .../flink-shaded-curator-test/pom.xml           |  2 +-
 flink-shaded-curator/pom.xml                    |  2 +-
 .../flink-shaded-hadoop1/pom.xml                |  2 +-
 .../flink-shaded-hadoop2/pom.xml                |  2 +-
 .../flink-shaded-include-yarn-tests/pom.xml     |  2 +-
 flink-shaded-hadoop/pom.xml                     |  2 +-
 .../flink-connector-elasticsearch/pom.xml       |  2 +-
 .../flink-connector-filesystem/pom.xml          |  2 +-
 .../flink-connector-flume/pom.xml               |  2 +-
 .../flink-connector-kafka-0.8/pom.xml           |  2 +-
 .../flink-connector-kafka-0.9/pom.xml           |  2 +-
 .../flink-connector-kafka-base/pom.xml          |  2 +-
 .../flink-connector-nifi/pom.xml                |  2 +-
 .../flink-connector-rabbitmq/pom.xml            |  2 +-
 .../flink-connector-twitter/pom.xml             |  2 +-
 flink-streaming-connectors/pom.xml              |  2 +-
 flink-streaming-java/pom.xml                    |  2 +-
 flink-streaming-scala/pom.xml                   |  2 +-
 flink-test-utils/pom.xml                        |  2 +-
 flink-tests/pom.xml                             |  2 +-
 flink-yarn-tests/pom.xml                        |  2 +-
 flink-yarn/pom.xml                              |  2 +-
 pom.xml                                         |  4 +-
 tools/force-shading/pom.xml                     |  2 +-
 71 files changed, 70 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
deleted file mode 100644
index 629d62c..0000000
--- a/.gitignore
+++ /dev/null
@@ -1,27 +0,0 @@
-.cache
-scalastyle-output.xml
-.classpath
-.idea
-.metadata
-.settings
-.project
-.version.properties
-filter.properties
-logs.zip
-target
-tmp
-*.class
-*.iml
-*.swp
-*.jar
-*.log
-.DS_Store
-_site
-docs/api
-build-target
-flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/
-flink-runtime-web/web-dashboard/assets/fonts/
-flink-runtime-web/web-dashboard/node_modules/
-flink-runtime-web/web-dashboard/bower_components/
-atlassian-ide-plugin.xml
-out/

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
deleted file mode 100644
index 0810fb7..0000000
--- a/.travis.yml
+++ /dev/null
@@ -1,57 +0,0 @@
-# s3 deployment based on http://about.travis-ci.org/blog/2012-12-18-travis-artifacts/
-
-# send to container based infrastructure: http://docs.travis-ci.com/user/workers/container-based-infrastructure/
-sudo: false
-
-cache:
-  directories:
-  - $HOME/.m2
-
-install: true
-
-language: java
-
-#See https://issues.apache.org/jira/browse/FLINK-1072
-# NOTE: When changing the build matrix, please also check with the tools/deploy_to_maven.sh file
-# The file assumes a certain build order for the maven / nigthly build deployments.
-matrix:
-  include:
-    - jdk: "oraclejdk8"
-      env: PROFILE="-Dhadoop.version=2.7.2 -Dscala-2.11 -Pinclude-yarn-tests -Dmaven.javadoc.skip=true"
-    - jdk: "oraclejdk8"
-      env: PROFILE="-Dhadoop.version=2.6.3 -Pinclude-yarn-tests -Dmaven.javadoc.skip=true"
-    - jdk: "openjdk7"
-      env: PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.11 -Pinclude-yarn-tests"
-    - jdk: "oraclejdk7" # this uploads the Hadoop 2 build to Maven and S3
-      env: PROFILE="-Dhadoop.version=2.3.0"
-    - jdk: "openjdk7" # this uploads the Hadoop 1 build to Maven and S3
-      env: PROFILE="-Dhadoop.profile=1"
-
-
-git:
-  depth: 100
-
-notifications:
-  webhooks:
-    urls:
-      - https://webhooks.gitter.im/e/d70a7e674cb9354c77b2
-    on_success: always  # options: [always|never|change] default: always
-    on_failure: always  # options: [always|never|change] default: always
-  slack:
-    secure: iYjxJn8OkCRslJ30/PcE+EbMiqfKwsvUJiVUEQAEXqCEwZg+wYDsN0ilPQQT0zU16mYWKoMTx71zrOZpjirGq7ww0XZ0wAfXDjgmTxX/DaEdp87uNgTRdQzLV7mQouMKZni28eoa08Rb2NIoLLQ39q7uCu0W/p7vAD2e9xHlBBE=
-
-env:
-    global:
-        # Global variable to avoid hanging travis builds when downloading cache archives.
-        - MALLOC_ARENA_MAX=2
-        # Build artifacts like logs (variables for apache/flink repo)
-        - secure: "Fm3NK28qN8yLtpJl4VI58biBECpOodMYbYXPVWwa62R7jkhHl2U1s4Xa5ujEgNIDcsUsY66z0V4pU0Es0XLNOY2ajlaFOHTmngzFIXul1r4vuNy0H8okEBjs9Ks0TOWYrE6ndAv1J4/oUsRtehayrriaehn31emXL9c4RSKgaiQ="
-        - secure: "CGcWDpoPLKVPVxFCa+rh5svyrSy7tWTsydsFuLlw5BH5QR57FWH0P5ZBZ31MPppoNNpKEp1V5PBxOH0dUAx8SVNWQFNCsQrOwVpTnTlyl3Cd1udj2hahbB3l+IGf0+O3v2vv6blYm6vJb98NqzZknjdIefDDBfu52ndJy1UqHQw="
-        - secure: "J4IJ7ZG5X+x/2K00kCpj6N/j3wEc6vG59KdDFVZp1WnKH8H0cty2wujZvDhBV+krbqja2MHhXQt/2mDjqm7pkdk1YElDOWsx909aw29wUdDN4yOsxFekIa5jMCrcQxbwzDRal6JmAzCakk51qIEgCYuAKquT0N+oETmnOhmcQe0="
-
-before_script:
-   - "gem install --no-document --version 0.8.9 faraday "
-
-# We run mvn and monitor its output. If there is no output for the specified number of seconds, we
-# print the stack traces of all running Java processes.
-script: "./tools/travis_mvn_watchdog.sh 300"

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/deploysettings.xml
----------------------------------------------------------------------
diff --git a/deploysettings.xml b/deploysettings.xml
deleted file mode 100644
index e36d848..0000000
--- a/deploysettings.xml
+++ /dev/null
@@ -1,38 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
-
--->
-
-<settings xmlns="http://maven.apache.org/SETTINGS/1.0.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/SETTINGS/1.0.0
-                      http://maven.apache.org/xsd/settings-1.0.0.xsd">
-  <servers>
-    <server>
-      <id>apache.snapshots.https</id>
-      <username>${sonatype_user}</username>
-      <password>${sonatype_pw}</password>
-    </server>
-    <server>
-      <id>apache.releases.https</id>
-      <username>${sonatype_user}</username>
-      <password>${sonatype_pw}</password>
-    </server>
-  </servers>
-</settings>
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 494eb2c..58f646e 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.0.2"
-version_short: "1.0" # Used for the top navbar w/o snapshot suffix
+version: 1.0.3
+version_short: 1.0.3
 
 # This suffix is appended to the Scala-dependent Maven artifact names
 scala_version_suffix: "_2.10"

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-annotations/pom.xml
----------------------------------------------------------------------
diff --git a/flink-annotations/pom.xml b/flink-annotations/pom.xml
index 05d9b0c..2c320d4 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-batch-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/pom.xml b/flink-batch-connectors/flink-avro/pom.xml
index c4f7ac6..af745dd 100644
--- a/flink-batch-connectors/flink-avro/pom.xml
+++ b/flink-batch-connectors/flink-avro/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/pom.xml b/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
index 9ccd61a..0cebdd3 100644
--- a/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
+++ b/flink-batch-connectors/flink-hadoop-compatibility/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-batch-connectors/flink-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/pom.xml b/flink-batch-connectors/flink-hbase/pom.xml
index ad6df4b..665794f 100644
--- a/flink-batch-connectors/flink-hbase/pom.xml
+++ b/flink-batch-connectors/flink-hbase/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-batch-connectors/flink-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hcatalog/pom.xml b/flink-batch-connectors/flink-hcatalog/pom.xml
index f1169a8..593f287 100644
--- a/flink-batch-connectors/flink-hcatalog/pom.xml
+++ b/flink-batch-connectors/flink-hcatalog/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-batch-connectors/flink-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-jdbc/pom.xml b/flink-batch-connectors/flink-jdbc/pom.xml
index 20e5bdb..d42cdad 100644
--- a/flink-batch-connectors/flink-jdbc/pom.xml
+++ b/flink-batch-connectors/flink-jdbc/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-batch-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-batch-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/pom.xml b/flink-batch-connectors/pom.xml
index 25f98d1..b915538 100644
--- a/flink-batch-connectors/pom.xml
+++ b/flink-batch-connectors/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-clients/pom.xml
----------------------------------------------------------------------
diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml
index 344fde6..f0b189e 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 1ae4cb8..4dd2cd3 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-contrib/flink-operator-stats/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-operator-stats/pom.xml b/flink-contrib/flink-operator-stats/pom.xml
index 2e4230e..49121ab 100644
--- a/flink-contrib/flink-operator-stats/pom.xml
+++ b/flink-contrib/flink-operator-stats/pom.xml
@@ -24,7 +24,7 @@ under the License.
     <parent>
         <groupId>org.apache.flink</groupId>
         <artifactId>flink-contrib</artifactId>
-        <version>1.0-SNAPSHOT</version>
+        <version>1.0.3</version>
         <relativePath>..</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 9c1601e..a19edfb 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 921fa62..2bd3ca3 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-contrib/flink-storm/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml
index 6c5c339..9cdf9d6 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 88ecf20..b5b1162 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 2403f95..3d8e74f 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-contrib/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml
index 76f0f88..a3c877a 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index fc91156..d34b166 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index e97c1cf..1d6aba6 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 094dd92..6c956d3 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 3ea3276..57fc42d 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml
index a5b95fa..83e6139 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-fs-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml
index 020eb1d..1294301 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java/pom.xml b/flink-java/pom.xml
index 25cd0ff..be1f4e6 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-java8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml
index ec69887..737829e 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-libraries/flink-cep/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml
index 6f9faa9..6aa016c 100644
--- a/flink-libraries/flink-cep/pom.xml
+++ b/flink-libraries/flink-cep/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-libraries</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 96492ae..9edf4a9 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 c74aaeb..0bb959f 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.0-SNAPSHOT</version>
+        <version>1.0.3</version>
         <relativePath>..</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-libraries/flink-gelly/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml
index 0b3d362..bb52f64 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-libraries/flink-ml/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/pom.xml b/flink-libraries/flink-ml/pom.xml
index fe5b254..5d2fe71 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-libraries/flink-python/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-python/pom.xml b/flink-libraries/flink-python/pom.xml
index 7512e2f..f8f493f 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.0-SNAPSHOT</version>
+        <version>1.0.3</version>
         <relativePath>..</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-libraries/flink-table/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml
index 73f77f4..6da988f 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-libraries/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/pom.xml b/flink-libraries/pom.xml
index 64ad7e4..92b2133 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-optimizer/pom.xml
----------------------------------------------------------------------
diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml
index 0160ee5..3ff5acd 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 73e4d14..5f49412 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 07d2b0b..dc50547 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.0-SNAPSHOT</flink.version>
+		<flink.version>1.0.3</flink.version>
 	</properties>
 
 	<repositories>

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 4943bda..01413c1 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 d60d5de..133821d 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.0-SNAPSHOT</flink.version>
+		<flink.version>1.0.3</flink.version>
 	</properties>
 
 	<!-- 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-quickstart/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/pom.xml b/flink-quickstart/pom.xml
index 2fb287a..b2f7c33 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-runtime-web/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml
index 27bbbd5..ea4131b 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index c6b6d1d..b802307 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-scala-shell/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala-shell/pom.xml b/flink-scala-shell/pom.xml
index 95ce81d..c9fbba9 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index 5694e2f..1f632f7 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 81b812c..da9b527 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 52c2ba8..58dc803 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-shaded-curator/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-curator/pom.xml b/flink-shaded-curator/pom.xml
index 5184d2d..27fc904 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml
index e8634f5..af4bf49 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop1/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-shaded-hadoop</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 cf95198..367cfe2 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/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 96d05ae..867c86f 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-shaded-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml
index d0114f2..10b8ab8 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml b/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
index ea3964c..cae93dc 100644
--- a/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/pom.xml b/flink-streaming-connectors/flink-connector-filesystem/pom.xml
index b621e86..e0d097c 100644
--- a/flink-streaming-connectors/flink-connector-filesystem/pom.xml
+++ b/flink-streaming-connectors/flink-connector-filesystem/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-flume/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-flume/pom.xml b/flink-streaming-connectors/flink-connector-flume/pom.xml
index 02a0e47..46dbae7 100644
--- a/flink-streaming-connectors/flink-connector-flume/pom.xml
+++ b/flink-streaming-connectors/flink-connector-flume/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
index 5bfe2c4..f8d162f 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
index 21d319c..0b8a6d0 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
index 8104a79..9bc612d 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
+++ b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-nifi/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/pom.xml b/flink-streaming-connectors/flink-connector-nifi/pom.xml
index 47c4e6e..a7ee4be 100644
--- a/flink-streaming-connectors/flink-connector-nifi/pom.xml
+++ b/flink-streaming-connectors/flink-connector-nifi/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml b/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
index d27bee0..c594280 100644
--- a/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
+++ b/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/flink-connector-twitter/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/pom.xml b/flink-streaming-connectors/flink-connector-twitter/pom.xml
index a1a7034..8fbd15d 100644
--- a/flink-streaming-connectors/flink-connector-twitter/pom.xml
+++ b/flink-streaming-connectors/flink-connector-twitter/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-streaming-connectors</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/pom.xml b/flink-streaming-connectors/pom.xml
index fceae25..e8a1615 100644
--- a/flink-streaming-connectors/pom.xml
+++ b/flink-streaming-connectors/pom.xml
@@ -26,7 +26,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml
index 60e305d..066739e 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-streaming-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml
index f79cce9..9f8634b 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils/pom.xml b/flink-test-utils/pom.xml
index b9b81a6..b921b1b 100644
--- a/flink-test-utils/pom.xml
+++ b/flink-test-utils/pom.xml
@@ -25,7 +25,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index c7b1dc6..a6ff111 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-yarn-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index c2865db..40b3ead 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/flink-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml
index fcb3dc6..643897d 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.0-SNAPSHOT</version>
+		<version>1.0.3</version>
 		<relativePath>..</relativePath>
 	</parent>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c62d177..bfa9a0a 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.0-SNAPSHOT</version>
+	<version>1.0.3</version>
 
 	<name>flink</name>
 	<packaging>pom</packaging>
@@ -110,7 +110,7 @@ under the License.
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>force-shading</artifactId>
-			<version>1.0-SNAPSHOT</version>
+			<version>1.0.3</version>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/06c5d765/tools/force-shading/pom.xml
----------------------------------------------------------------------
diff --git a/tools/force-shading/pom.xml b/tools/force-shading/pom.xml
index 5283547..6bbd2de 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.0-SNAPSHOT</version>
+	<version>1.0.3</version>
 
 	<packaging>jar</packaging>