flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rmetz...@apache.org
Subject [5/5] flink git commit: [FLINK-3093] Introduce annotations for interface stability in remaining modules
Date Fri, 05 Feb 2016 13:44:30 GMT
[FLINK-3093] Introduce annotations for interface stability in remaining modules

This closes #1428


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

Branch: refs/heads/master
Commit: b54499b1cf6d14d01e448ea9ce8edabaaba83929
Parents: 2eb2a0e
Author: Robert Metzger <rmetzger@apache.org>
Authored: Tue Dec 1 19:32:30 2015 +0100
Committer: Robert Metzger <rmetzger@apache.org>
Committed: Fri Feb 5 14:43:49 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/annotation/Experimental.java   |  3 +-
 .../org/apache/flink/annotation/Internal.java   |  3 +-
 .../org/apache/flink/annotation/Public.java     |  1 +
 .../flink/api/java/io/AvroInputFormat.java      |  1 -
 .../flink/api/java/io/AvroOutputFormat.java     |  3 -
 .../mapred/HadoopMapFunction.java               |  2 +
 .../mapred/HadoopReduceCombineFunction.java     |  2 +
 .../mapred/HadoopReduceFunction.java            |  2 +
 .../flink/addons/hbase/TableInputFormat.java    |  2 -
 .../api/common/functions/RuntimeContext.java    |  5 ++
 .../util/AbstractRuntimeUDFContext.java         |  6 ++
 .../api/common/io/GenericCsvInputFormat.java    |  2 -
 .../common/io/statistics/BaseStatistics.java    |  7 ++
 .../SingleInputSemanticProperties.java          |  2 -
 .../flink/api/common/typeinfo/AtomicType.java   |  2 +
 .../api/common/typeinfo/BasicArrayTypeInfo.java | 15 +++-
 .../api/common/typeinfo/BasicTypeInfo.java      | 15 +++-
 .../api/common/typeinfo/FractionalTypeInfo.java |  2 +
 .../api/common/typeinfo/IntegerTypeInfo.java    |  2 +
 .../api/common/typeinfo/NothingTypeInfo.java    | 10 +++
 .../api/common/typeinfo/NumericTypeInfo.java    |  2 +
 .../common/typeinfo/PrimitiveArrayTypeInfo.java | 14 +++
 .../api/common/typeinfo/TypeInformation.java    | 10 +++
 .../api/common/typeutils/CompositeType.java     | 23 ++++-
 .../flink/api/java/functions/KeySelector.java   |  2 +
 .../org/apache/flink/api/java/tuple/Tuple.java  |  2 +
 .../org/apache/flink/api/java/tuple/Tuple0.java |  3 +
 .../org/apache/flink/api/java/tuple/Tuple1.java |  2 +
 .../apache/flink/api/java/tuple/Tuple10.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple11.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple12.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple13.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple14.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple15.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple16.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple17.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple18.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple19.java    |  2 +
 .../org/apache/flink/api/java/tuple/Tuple2.java |  2 +
 .../apache/flink/api/java/tuple/Tuple20.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple21.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple22.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple23.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple24.java    |  2 +
 .../apache/flink/api/java/tuple/Tuple25.java    |  2 +
 .../org/apache/flink/api/java/tuple/Tuple3.java |  2 +
 .../org/apache/flink/api/java/tuple/Tuple4.java |  2 +
 .../org/apache/flink/api/java/tuple/Tuple5.java |  2 +
 .../org/apache/flink/api/java/tuple/Tuple6.java |  2 +
 .../org/apache/flink/api/java/tuple/Tuple7.java |  2 +
 .../org/apache/flink/api/java/tuple/Tuple8.java |  2 +
 .../org/apache/flink/api/java/tuple/Tuple9.java |  2 +
 .../api/java/tuple/builder/Tuple0Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple10Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple11Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple12Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple13Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple14Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple15Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple16Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple17Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple18Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple19Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple1Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple20Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple21Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple22Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple23Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple24Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple25Builder.java  |  2 +
 .../api/java/tuple/builder/Tuple2Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple3Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple4Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple5Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple6Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple7Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple8Builder.java   |  2 +
 .../api/java/tuple/builder/Tuple9Builder.java   |  2 +
 .../flink/api/java/typeutils/AvroTypeInfo.java  |  4 +
 .../api/java/typeutils/EitherTypeInfo.java      | 11 +++
 .../flink/api/java/typeutils/EnumTypeInfo.java  | 12 +++
 .../api/java/typeutils/GenericTypeInfo.java     | 12 +++
 .../java/typeutils/InputTypeConfigurable.java   |  2 +
 .../api/java/typeutils/ObjectArrayTypeInfo.java | 16 +++-
 .../flink/api/java/typeutils/PojoTypeInfo.java  | 16 ++++
 .../api/java/typeutils/ResultTypeQueryable.java |  2 +
 .../flink/api/java/typeutils/TupleTypeInfo.java | 10 +++
 .../flink/api/java/typeutils/TypeExtractor.java | 93 +++++++++++++-------
 .../api/java/typeutils/TypeInfoParser.java      |  3 +-
 .../flink/api/java/typeutils/ValueTypeInfo.java | 18 +++-
 .../api/java/typeutils/WritableTypeInfo.java    | 17 +++-
 .../java/org/apache/flink/types/Either.java     |  3 +
 .../java/org/apache/flink/types/Nothing.java    |  5 +-
 .../flink/api/java/tuple/TupleGenerator.java    |  4 +
 flink-java/pom.xml                              |  2 +-
 .../java/org/apache/flink/api/java/DataSet.java |  6 ++
 .../flink/api/java/ExecutionEnvironment.java    | 24 +++++
 .../api/java/ExecutionEnvironmentFactory.java   |  3 +
 .../apache/flink/api/java/LocalEnvironment.java |  4 +
 .../flink/api/java/RemoteEnvironment.java       |  4 +
 .../api/java/aggregation/Aggregations.java      |  6 +-
 .../api/java/functions/FunctionAnnotation.java  | 11 ++-
 .../java/hadoop/mapred/HadoopInputFormat.java   |  2 +
 .../java/hadoop/mapred/HadoopOutputFormat.java  |  2 +
 .../hadoop/mapreduce/HadoopInputFormat.java     |  2 +
 .../hadoop/mapreduce/HadoopOutputFormat.java    |  2 +
 .../org/apache/flink/api/java/io/CsvReader.java |  4 +
 .../api/java/io/DiscardingOutputFormat.java     |  2 +
 .../flink/api/java/io/TextInputFormat.java      |  1 -
 .../flink/api/java/io/TextOutputFormat.java     |  1 -
 .../api/java/operators/AggregateOperator.java   |  5 ++
 .../java/operators/BulkIterationResultSet.java  |  2 +
 .../api/java/operators/CoGroupOperator.java     |  6 ++
 .../flink/api/java/operators/CrossOperator.java | 13 ++-
 .../java/operators/CustomUnaryOperation.java    |  2 +
 .../flink/api/java/operators/DataSink.java      | 16 +++-
 .../flink/api/java/operators/DataSource.java    |  6 ++
 .../api/java/operators/DeltaIteration.java      |  7 ++
 .../java/operators/DeltaIterationResultSet.java | 12 ++-
 .../api/java/operators/DistinctOperator.java    |  4 +
 .../api/java/operators/FilterOperator.java      |  2 +
 .../api/java/operators/FlatMapOperator.java     |  2 +
 .../java/operators/GroupCombineOperator.java    |  4 +
 .../api/java/operators/GroupReduceOperator.java |  6 +-
 .../flink/api/java/operators/Grouping.java      |  8 +-
 .../api/java/operators/IterativeDataSet.java    |  9 +-
 .../flink/api/java/operators/JoinOperator.java  | 20 ++++-
 .../flink/api/java/operators/MapOperator.java   |  2 +
 .../java/operators/MapPartitionOperator.java    |  2 +
 .../flink/api/java/operators/Operator.java      |  2 +
 .../api/java/operators/PartitionOperator.java   |  4 +
 .../api/java/operators/ProjectOperator.java     |  8 +-
 .../api/java/operators/ReduceOperator.java      |  4 +
 .../api/java/operators/SingleInputOperator.java |  2 +
 .../java/operators/SingleInputUdfOperator.java  |  6 ++
 .../java/operators/SortPartitionOperator.java   |  2 +
 .../api/java/operators/SortedGrouping.java      |  2 +
 .../api/java/operators/TwoInputOperator.java    |  2 +
 .../api/java/operators/TwoInputUdfOperator.java |  8 +-
 .../flink/api/java/operators/UdfOperator.java   |  5 ++
 .../flink/api/java/operators/UnionOperator.java |  2 +
 .../api/java/operators/UnsortedGrouping.java    |  2 +
 .../operators/join/JoinFunctionAssigner.java    |  2 +
 .../operators/join/JoinOperatorSetsBase.java    |  2 +
 .../flink/api/java/operators/join/JoinType.java |  3 +
 .../api/java/sampling/BernoulliSampler.java     |  2 +-
 .../flink/api/java/utils/DataSetUtils.java      |  2 +
 .../flink/api/java/utils/ParameterTool.java     |  4 +
 .../jobgraph/tasks/InputSplitProvider.java      |  2 +
 .../flink/runtime/state/CheckpointListener.java |  1 +
 .../apache/flink/runtime/state/StateHandle.java |  1 +
 .../scala/operators/ScalaAggregateOperator.java |  4 +
 .../flink/api/scala/AggregateDataSet.scala      |  2 +
 .../apache/flink/api/scala/CoGroupDataSet.scala |  3 +
 .../apache/flink/api/scala/CrossDataSet.scala   |  2 +
 .../org/apache/flink/api/scala/DataSet.scala    |  7 +-
 .../flink/api/scala/ExecutionEnvironment.scala  | 18 ++++
 .../apache/flink/api/scala/GroupedDataSet.scala |  3 +
 .../api/scala/PartitionSortedDataSet.scala      |  2 +
 .../api/scala/UnfinishedCoGroupOperation.scala  |  6 +-
 .../scala/hadoop/mapred/HadoopInputFormat.scala |  2 +
 .../hadoop/mapred/HadoopOutputFormat.scala      |  2 +
 .../hadoop/mapreduce/HadoopInputFormat.scala    |  2 +
 .../hadoop/mapreduce/HadoopOutputFormat.scala   |  2 +
 .../apache/flink/api/scala/joinDataSet.scala    |  8 ++
 .../api/scala/typeutils/CaseClassTypeInfo.scala |  9 ++
 .../api/scala/typeutils/EitherTypeInfo.scala    | 10 +++
 .../api/scala/typeutils/EnumValueTypeInfo.scala | 11 +++
 .../api/scala/typeutils/OptionTypeInfo.scala    | 10 +++
 .../scala/typeutils/ScalaNothingTypeInfo.scala  |  9 ++
 .../scala/typeutils/TraversableTypeInfo.scala   | 10 +++
 .../flink/api/scala/typeutils/TryTypeInfo.scala | 10 +++
 .../api/scala/typeutils/UnitTypeInfo.scala      |  9 ++
 .../apache/flink/api/scala/utils/package.scala  |  3 +
 .../flink/streaming/connectors/fs/Clock.java    |  1 +
 .../streaming/connectors/fs/SystemClock.java    |  1 +
 .../flink/streaming/api/CheckpointingMode.java  |  3 +
 .../flink/streaming/api/TimeCharacteristic.java |  1 +
 .../checkpoint/CheckpointedAsynchronously.java  |  1 +
 .../api/datastream/AllWindowedStream.java       |  6 ++
 .../api/datastream/CoGroupedStreams.java        | 11 +++
 .../api/datastream/ConnectedStreams.java        |  4 +
 .../streaming/api/datastream/DataStream.java    | 28 ++++++
 .../api/datastream/DataStreamSink.java          |  7 ++
 .../api/datastream/DataStreamSource.java        |  2 +
 .../api/datastream/IterativeStream.java         |  4 +
 .../streaming/api/datastream/JoinedStreams.java | 16 +++-
 .../streaming/api/datastream/KeyedStream.java   |  8 ++
 .../datastream/SingleOutputStreamOperator.java  | 12 +++
 .../streaming/api/datastream/SplitStream.java   |  3 +
 .../api/datastream/WindowedStream.java          |  6 ++
 .../api/environment/CheckpointConfig.java       |  5 ++
 .../api/environment/LocalStreamEnvironment.java |  2 +
 .../environment/RemoteStreamEnvironment.java    |  2 +
 .../environment/StreamExecutionEnvironment.java | 24 +++++
 .../api/functions/co/CoFlatMapFunction.java     |  2 +
 .../api/functions/co/CoMapFunction.java         |  2 +
 .../api/functions/co/RichCoFlatMapFunction.java |  2 +
 .../api/functions/co/RichCoMapFunction.java     |  2 +
 .../api/functions/sink/DiscardingSink.java      |  3 +
 .../api/functions/sink/RichSinkFunction.java    |  2 +
 .../api/functions/sink/SinkFunction.java        |  2 +
 .../source/EventTimeSourceFunction.java         |  3 +
 .../source/ParallelSourceFunction.java          |  3 +
 .../source/RichParallelSourceFunction.java      |  2 +
 .../functions/source/RichSourceFunction.java    |  2 +
 .../api/functions/source/SourceFunction.java    |  6 ++
 .../functions/windowing/AllWindowFunction.java  |  4 +-
 .../windowing/RichAllWindowFunction.java        |  2 +
 .../functions/windowing/RichWindowFunction.java |  2 +
 .../api/functions/windowing/WindowFunction.java |  2 +
 .../api/operators/AbstractStreamOperator.java   |  2 +-
 .../api/operators/ChainingStrategy.java         |  1 +
 .../streaming/api/windowing/time/Time.java      |  3 +-
 .../streaming/api/windowing/windows/Window.java |  1 +
 .../runtime/streamrecord/StreamRecord.java      |  1 +
 .../serialization/DeserializationSchema.java    |  2 +
 .../util/serialization/SerializationSchema.java |  3 +
 .../TypeInformationSerializationSchema.java     |  2 +
 .../streaming/api/scala/AllWindowedStream.scala |  4 +
 .../streaming/api/scala/CoGroupedStreams.scala  |  6 ++
 .../streaming/api/scala/ConnectedStreams.scala  |  2 +
 .../flink/streaming/api/scala/DataStream.scala  | 29 ++++++
 .../streaming/api/scala/JoinedStreams.scala     |  6 +-
 .../flink/streaming/api/scala/KeyedStream.scala |  5 +-
 .../flink/streaming/api/scala/SplitStream.scala |  2 +
 .../api/scala/StreamExecutionEnvironment.scala  | 17 ++++
 .../streaming/api/scala/WindowedStream.scala    |  4 +
 .../api/scala/function/StatefulFunction.scala   |  2 +
 229 files changed, 1077 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-annotations/src/main/java/org/apache/flink/annotation/Experimental.java
----------------------------------------------------------------------
diff --git a/flink-annotations/src/main/java/org/apache/flink/annotation/Experimental.java b/flink-annotations/src/main/java/org/apache/flink/annotation/Experimental.java
index bf9a97c..3f4a661 100644
--- a/flink-annotations/src/main/java/org/apache/flink/annotation/Experimental.java
+++ b/flink-annotations/src/main/java/org/apache/flink/annotation/Experimental.java
@@ -29,6 +29,7 @@ import java.lang.annotation.Target;
  * An experimental API might change between minor releases.
  */
 @Documented
-@Target({ ElementType.TYPE, ElementType.METHOD })
+@Target({ ElementType.TYPE, ElementType.METHOD, ElementType.FIELD, ElementType.CONSTRUCTOR })
+@Public
 public @interface Experimental {
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java
----------------------------------------------------------------------
diff --git a/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java b/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java
index 4df6847..9de1aba 100644
--- a/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java
+++ b/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java
@@ -28,6 +28,7 @@ import java.lang.annotation.Target;
  * Developer APIs are stable but internal to Flink and might change across releases.
  */
 @Documented
-@Target({ ElementType.TYPE, ElementType.METHOD })
+@Target({ ElementType.TYPE, ElementType.METHOD, ElementType.CONSTRUCTOR })
+@Public
 public @interface Internal {
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-annotations/src/main/java/org/apache/flink/annotation/Public.java
----------------------------------------------------------------------
diff --git a/flink-annotations/src/main/java/org/apache/flink/annotation/Public.java b/flink-annotations/src/main/java/org/apache/flink/annotation/Public.java
index 28c4d83..e545eef 100644
--- a/flink-annotations/src/main/java/org/apache/flink/annotation/Public.java
+++ b/flink-annotations/src/main/java/org/apache/flink/annotation/Public.java
@@ -32,4 +32,5 @@ import java.lang.annotation.Target;
  */
 @Documented
 @Target(ElementType.TYPE)
+@Public
 public @interface Public {}

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
index 09fcacb..9457b98 100644
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
+++ b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroInputFormat.java
@@ -38,7 +38,6 @@ import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.util.InstantiationUtil;
 
-
 public class AvroInputFormat<E> extends FileInputFormat<E> implements ResultTypeQueryable<E> {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
index d00dbf7..9a3a025 100644
--- a/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
+++ b/flink-batch-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java
@@ -15,11 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.api.java.io;
 
-
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.io.DatumWriter;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
index 83ab23d..ba8aa90 100644
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
+++ b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
@@ -23,6 +23,7 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -42,6 +43,7 @@ import org.apache.hadoop.mapred.Reporter;
  * This wrapper maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction. 
  */
 @SuppressWarnings("rawtypes")
+@Public
 public final class HadoopMapFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
 					extends RichFlatMapFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYOUT,VALUEOUT>> 
 					implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
index e164d89..c1acc2b 100644
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
+++ b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
@@ -23,6 +23,7 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.functions.GroupCombineFunction;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -45,6 +46,7 @@ import org.apache.hadoop.mapred.Reporter;
  * This wrapper maps a Hadoop Reducer and Combiner (mapred API) to a combinable Flink GroupReduceFunction.
  */
 @SuppressWarnings("rawtypes")
+@Public
 public final class HadoopReduceCombineFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
 	extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>>
 	implements GroupCombineFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYIN,VALUEIN>>,

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
index 1c47696..55aea24 100644
--- a/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
+++ b/flink-batch-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
@@ -23,6 +23,7 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -44,6 +45,7 @@ import org.apache.hadoop.mapred.Reporter;
  * This wrapper maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction. 
  */
 @SuppressWarnings("rawtypes")
+@Public
 public final class HadoopReduceFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT> 
 					extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>> 
 					implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java b/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
index ca1eb72..6ba6217 100644
--- a/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
+++ b/flink-batch-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.addons.hbase;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
index 92dd518..c96ecde 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
@@ -239,6 +239,7 @@ public interface RuntimeContext {
 	 * @throws UnsupportedOperationException Thrown, if no partitioned state is available for the
 	 *                                       function (function is not part of a KeyedStream).
 	 */
+	@Experimental
 	<T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties);
 
 	/**
@@ -282,6 +283,7 @@ public interface RuntimeContext {
 	 * @throws UnsupportedOperationException Thrown, if no partitioned state is available for the
 	 *                                       function (function is not part os a KeyedStream).
 	 */
+	@Experimental
 	<T> ListState<T> getListState(ListStateDescriptor<T> stateProperties);
 
 	/**
@@ -321,6 +323,7 @@ public interface RuntimeContext {
 	 * @throws UnsupportedOperationException Thrown, if no partitioned state is available for the
 	 *                                       function (function is not part of a KeyedStream).
 	 */
+	@Experimental
 	<T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties);
 	
 	/**
@@ -380,6 +383,7 @@ public interface RuntimeContext {
 	 * @deprecated Use the more expressive {@link #getState(ValueStateDescriptor)} instead.
 	 */
 	@Deprecated
+	@Experimental
 	<S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState);
 
 	/**
@@ -432,5 +436,6 @@ public interface RuntimeContext {
 	 * @deprecated Use the more expressive {@link #getState(ValueStateDescriptor)} instead.
 	 */
 	@Deprecated
+	@Experimental
 	<S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
index e0b53f6..a8078fb 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.concurrent.Future;
 
 import com.google.common.base.Preconditions;
+import org.apache.flink.annotation.Experimental;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.api.common.accumulators.Accumulator;
@@ -175,18 +176,21 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext {
 	}
 
 	@Override
+	@Experimental
 	public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
 		throw new UnsupportedOperationException(
 				"This state is only accessible by functions executed on a KeyedStream");
 	}
 
 	@Override
+	@Experimental
 	public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
 		throw new UnsupportedOperationException(
 				"This state is only accessible by functions executed on a KeyedStream");
 	}
 
 	@Override
+	@Experimental
 	public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
 		throw new UnsupportedOperationException(
 				"This state is only accessible by functions executed on a KeyedStream");
@@ -194,6 +198,7 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext {
 
 	@Override
 	@Deprecated
+	@Experimental
 	public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
 		throw new UnsupportedOperationException(
 				"This state is only accessible by functions executed on a KeyedStream");
@@ -201,6 +206,7 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext {
 
 	@Override
 	@Deprecated
+	@Experimental
 	public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
 		throw new UnsupportedOperationException(
 				"This state is only accessible by functions executed on a KeyedStream");

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/io/GenericCsvInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/GenericCsvInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/GenericCsvInputFormat.java
index ddbf165..89caf2f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/GenericCsvInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/GenericCsvInputFormat.java
@@ -23,7 +23,6 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 
-import org.apache.flink.annotation.Public;
 import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.types.parser.FieldParser;
@@ -41,7 +40,6 @@ import java.util.ArrayList;
 import java.util.Map;
 import java.util.TreeMap;
 
-@Public
 public abstract class GenericCsvInputFormat<OT> extends DelimitedInputFormat<OT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(GenericCsvInputFormat.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/io/statistics/BaseStatistics.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/statistics/BaseStatistics.java b/flink-core/src/main/java/org/apache/flink/api/common/io/statistics/BaseStatistics.java
index ef92f18..cb194ae 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/statistics/BaseStatistics.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/statistics/BaseStatistics.java
@@ -19,6 +19,7 @@
 
 package org.apache.flink.api.common.io.statistics;
 
+import org.apache.flink.annotation.Experimental;
 import org.apache.flink.annotation.Public;
 
 /**
@@ -30,16 +31,19 @@ public interface BaseStatistics {
 	/**
 	 * Constant indicating that the input size is unknown.
 	 */
+	@Experimental
 	public static final long SIZE_UNKNOWN = -1;
 	
 	/**
 	 * Constant indicating that the number of records is unknown;
 	 */
+	@Experimental
 	public static final long NUM_RECORDS_UNKNOWN = -1;
 	
 	/**
 	 * Constant indicating that average record width is unknown.
 	 */
+	@Experimental
 	public static final float AVG_RECORD_BYTES_UNKNOWN = -1.0f;
 	
 	// --------------------------------------------------------------------------------------------
@@ -49,6 +53,7 @@ public interface BaseStatistics {
 	 *   
 	 * @return The total size of the input, in bytes.
 	 */
+	@Experimental
 	public long getTotalInputSize();
 	
 	/**
@@ -56,6 +61,7 @@ public interface BaseStatistics {
 	 * 
 	 * @return The number of records in the input.
 	 */
+	@Experimental
 	public long getNumberOfRecords();
 	
 	/**
@@ -63,5 +69,6 @@ public interface BaseStatistics {
 	 * 
 	 * @return The average width of a record in bytes.
 	 */
+	@Experimental
 	public float getAverageRecordWidth();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputSemanticProperties.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputSemanticProperties.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputSemanticProperties.java
index db27a9e..23bbc8b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputSemanticProperties.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputSemanticProperties.java
@@ -21,13 +21,11 @@ package org.apache.flink.api.common.operators;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.operators.util.FieldSet;
 
 /**
  * Container for the semantic properties associated to a single input operator.
  */
-@Public
 public class SingleInputSemanticProperties implements SemanticProperties {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java
index 35e749d..ccd36c0 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.common.typeinfo;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 
@@ -29,6 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
  * In contrast to atomic types are composite types, where the type information is aware of the individual
  * fields and individual fields may be used as a key.
  */
+@Public
 public interface AtomicType<T> {
 	
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
index c72e8ed..995c2fe 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
@@ -23,12 +23,15 @@ import java.util.Map;
 import java.util.Objects;
 
 import com.google.common.base.Preconditions;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.array.StringArraySerializer;
 import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.typeutils.base.GenericArraySerializer;
 
+@Public
 public final class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
 
 	private static final long serialVersionUID = 1L;
@@ -58,45 +61,54 @@ public final class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
 	// --------------------------------------------------------------------------------------------
 
 	@Override
+	@Experimental
 	public boolean isBasicType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public boolean isTupleType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public int getArity() {
 		return 1;
 	}
 	
 	@Override
+	@Experimental
 	public int getTotalFields() {
 		return 1;
 	}
 
 	@Override
+	@Experimental
 	public Class<T> getTypeClass() {
 		return this.arrayClass;
 	}
 
+	@Experimental
 	public Class<C> getComponentTypeClass() {
 		return this.componentInfo.getTypeClass();
 	}
-	
+
+	@Experimental
 	public TypeInformation<C> getComponentInfo() {
 		return componentInfo;
 	}
 
 	@Override
+	@Experimental
 	public boolean isKeyType() {
 		return false;
 	}
 
 	@Override
 	@SuppressWarnings("unchecked")
+	@Experimental
 	public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
 		// special case the string array
 		if (componentInfo.getTypeClass().equals(String.class)) {
@@ -140,6 +152,7 @@ public final class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
 	// --------------------------------------------------------------------------------------------
 
 	@SuppressWarnings("unchecked")
+	@Experimental
 	public static <X, C> BasicArrayTypeInfo<X, C> getInfoFor(Class<X> type) {
 		if (!type.isArray()) {
 			throw new InvalidTypesException("The given class is no array.");

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
index 3e34644..0b6b067 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
@@ -26,6 +26,8 @@ import java.util.Map;
 import java.util.Objects;
 
 import com.google.common.base.Preconditions;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.typeutils.TypeComparator;
@@ -56,6 +58,7 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer;
 /**
  * Type information for primitive types (int, long, double, byte, ...), String, Date, and Void.
  */
+@Public
 public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T> {
 
 	private static final long serialVersionUID = -430955220409131770L;
@@ -97,6 +100,7 @@ public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T
 	 * Returns whether this type should be automatically casted to
 	 * the target type in an arithmetic operation.
 	 */
+	@Experimental
 	public boolean shouldAutocastTo(BasicTypeInfo<?> to) {
 		for (Class<?> possibleTo: possibleCastTargetTypes) {
 			if (possibleTo.equals(to.getTypeClass())) {
@@ -107,41 +111,49 @@ public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T
 	}
 
 	@Override
+	@Experimental
 	public boolean isBasicType() {
 		return true;
 	}
 
 	@Override
+	@Experimental
 	public boolean isTupleType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public int getArity() {
 		return 1;
 	}
 
 	@Override
+	@Experimental
 	public int getTotalFields() {
 		return 1;
 	}
 	
 	@Override
+	@Experimental
 	public Class<T> getTypeClass() {
 		return this.clazz;
 	}
 	
 	@Override
+	@Experimental
 	public boolean isKeyType() {
 		return true;
 	}
 	
 	@Override
+	@Experimental
 	public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
 		return this.serializer;
 	}
 	
 	@Override
+	@Experimental
 	public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) {
 		if (comparatorClass != null) {
 			return instantiateComparator(comparatorClass, sortOrderAscending);
@@ -183,7 +195,8 @@ public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	
+
+	@Experimental
 	public static <X> BasicTypeInfo<X> getInfoFor(Class<X> type) {
 		if (type == null) {
 			throw new NullPointerException();

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
index f357e34..aa22ac6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.common.typeinfo;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
@@ -28,6 +29,7 @@ import java.util.Set;
 /**
  * Type information for numeric fractional primitive types (double, float).
  */
+@Public
 public class FractionalTypeInfo<T> extends NumericTypeInfo<T> {
 
 	private static final long serialVersionUID = 554334260950199994L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
index 7f9dd92..bff3ab7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.common.typeinfo;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
@@ -28,6 +29,7 @@ import java.util.Set;
 /**
  * Type information for numeric integer primitive types: int, long, byte, short, character.
  */
+@Public
 public class IntegerTypeInfo<T> extends NumericTypeInfo<T> {
 
 	private static final long serialVersionUID = -8068827354966766955L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
index a32c898..1e60265 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.api.common.typeinfo;
 
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.types.Nothing;
@@ -25,41 +27,49 @@ import org.apache.flink.types.Nothing;
 /**
  * Placeholder type information for the {@link Nothing} type.
  */
+@Public
 public class NothingTypeInfo extends TypeInformation<Nothing> {
 	
 	private static final long serialVersionUID = 1L;
 	
 	@Override
+	@Experimental
 	public boolean isBasicType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public boolean isTupleType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public int getArity() {
 		return 0;
 	}
 
 	@Override
+	@Experimental
 	public int getTotalFields() {
 		return 0;
 	}
 	
 	@Override
+	@Experimental
 	public Class<Nothing> getTypeClass() {
 		return Nothing.class;
 	}
 
 	@Override
+	@Experimental
 	public boolean isKeyType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public TypeSerializer<Nothing> createSerializer(ExecutionConfig executionConfig) {
 		throw new RuntimeException("The Nothing type cannot have a serializer.");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
index 830e297..6969520 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.common.typeinfo;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
@@ -28,6 +29,7 @@ import java.util.Set;
 /**
  * Type information for numeric primitive types: int, long, double, byte, short, float, char.
  */
+@Public
 public abstract class NumericTypeInfo<T> extends BasicTypeInfo<T> {
 
 	private static final long serialVersionUID = -5937777910658986986L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
index 9bb444a..a4cf434 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
@@ -23,6 +23,8 @@ import java.util.Map;
 import java.util.Objects;
 
 import com.google.common.base.Preconditions;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -50,6 +52,7 @@ import org.apache.flink.api.common.typeutils.base.array.ShortPrimitiveArraySeria
  *
  * @param <T> The type represented by this type information, e.g., int[], double[], long[]
  */
+@Public
 public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements AtomicType<T> {
 
 	private static final long serialVersionUID = 1L;
@@ -92,36 +95,43 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
 	// --------------------------------------------------------------------------------------------
 
 	@Override
+	@Experimental
 	public boolean isBasicType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public boolean isTupleType() {
 		return false;
 	}
 
 	@Override
+	@Experimental
 	public int getArity() {
 		return 1;
 	}
 	
 	@Override
+	@Experimental
 	public int getTotalFields() {
 		return 1;
 	}
 
 	@Override
+	@Experimental
 	public Class<T> getTypeClass() {
 		return this.arrayClass;
 	}
 
 	@Override
+	@Experimental
 	public boolean isKeyType() {
 		return true;
 	}
 
 	@Override
+	@Experimental
 	public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
 		return this.serializer;
 	}
@@ -130,6 +140,7 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
 	 * Gets the class that represents the component type.
 	 * @return The class of the component type.
 	 */
+	@Experimental
 	public Class<?> getComponentClass() {
 		return this.arrayClass.getComponentType();
 	}
@@ -138,6 +149,7 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
 	 * Gets the type information of the component type.
 	 * @return The type information of the component type.
 	 */
+	@Experimental
 	public TypeInformation<?> getComponentType() {
 		return BasicTypeInfo.getInfoFor(getComponentClass());
 	}
@@ -183,6 +195,7 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
 	 * @throws InvalidTypesException Thrown, if the given class does not represent an array.
 	 */
 	@SuppressWarnings("unchecked")
+	@Experimental
 	public static <X> PrimitiveArrayTypeInfo<X> getInfoFor(Class<X> type) {
 		if (!type.isArray()) {
 			throw new InvalidTypesException("The given class is no array.");
@@ -208,6 +221,7 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
 	}
 
 	@Override
+	@Experimental
 	public PrimitiveArrayComparator<T, ?> createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) {
 		try {
 			return comparatorClass.getConstructor(boolean.class).newInstance(sortOrderAscending);

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
index 3f4b196..8d96d34 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.common.typeinfo;
 
+import org.apache.flink.annotation.Experimental;
 import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -80,6 +81,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 *  
 	 * @return True, if this type information describes a basic type, false otherwise.
 	 */
+	@Experimental
 	public abstract boolean isBasicType();
 	
 	/**
@@ -88,6 +90,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 *  
 	 * @return True, if this type information describes a tuple type, false otherwise.
 	 */
+	@Experimental
 	public abstract boolean isTupleType();
 	
 	/**
@@ -95,6 +98,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 * 
 	 * @return Gets the number of fields in this type without nesting.
 	 */
+	@Experimental
 	public abstract int getArity();
 	
 	/**
@@ -105,6 +109,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 * 
 	 * @return The number of fields in this type, including its sub-fields (for composite types) 
 	 */
+	@Experimental
 	public abstract int getTotalFields();
 	
 	/**
@@ -112,6 +117,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 *  
 	 * @return The class of the type represented by this type information.
 	 */
+	@Experimental
 	public abstract Class<T> getTypeClass();
 
 	/**
@@ -119,6 +125,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 *
 	 * @return The list of generic parameters. This list can be empty.
 	 */
+	@Experimental
 	public List<TypeInformation<?>> getGenericParameters() {
 		// Return an empty list as the default implementation
 		return new LinkedList<>();
@@ -130,12 +137,14 @@ public abstract class TypeInformation<T> implements Serializable {
 	 *  
 	 * @return True, if the type can be used as a key, false otherwise.
 	 */
+	@Experimental
 	public abstract boolean isKeyType();
 
 	/**
 	 * Checks whether this type can be used as a key for sorting.
 	 * The order produced by sorting this type must be meaningful.
 	 */
+	@Experimental
 	public boolean isSortKeyType() {
 		return isKeyType();
 	}
@@ -147,6 +156,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 * @param config The config used to parameterize the serializer.
 	 * @return A serializer for this type.
 	 */
+	@Experimental
 	public abstract TypeSerializer<T> createSerializer(ExecutionConfig config);
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
index 909afdd..959750a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Objects;
 
 import com.google.common.base.Preconditions;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.AtomicType;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -33,12 +35,14 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
  * 
  * The class is taking care of serialization and comparators for Tuples as well.
  */
+@Public
 public abstract class CompositeType<T> extends TypeInformation<T> {
 	
 	private static final long serialVersionUID = 1L;
 	
 	private final Class<T> typeClass;
-	
+
+	@Experimental
 	public CompositeType(Class<T> typeClass) {
 		this.typeClass = Preconditions.checkNotNull(typeClass);
 	}
@@ -48,6 +52,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 *
 	 * @return Type class of the composite type
 	 */
+	@Experimental
 	public Class<T> getTypeClass() {
 		return typeClass;
 	}
@@ -58,6 +63,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * @param fieldExpression The field expression for which the flat field descriptors are computed.
 	 * @return The list of descriptors for the flat fields which are specified by the field expression.
 	 */
+	@Experimental
 	public List<FlatFieldDescriptor> getFlatFields(String fieldExpression) {
 		List<FlatFieldDescriptor> result = new ArrayList<FlatFieldDescriptor>();
 		this.getFlatFields(fieldExpression, 0, result);
@@ -71,6 +77,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * @param offset The offset to use when computing the positions of the flat fields.
 	 * @param result The list into which all flat field descriptors are inserted.
 	 */
+	@Experimental
 	public abstract void getFlatFields(String fieldExpression, int offset, List<FlatFieldDescriptor> result);
 
 	/**
@@ -80,6 +87,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * @param fieldExpression The field expression for which the field of which the type is returned.
 	 * @return The type of the field at the given field expression.
 	 */
+	@Experimental
 	public abstract <X> TypeInformation<X> getTypeAt(String fieldExpression);
 
 	/**
@@ -88,8 +96,10 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * @param pos The position of the (unnested) field in this composite type.
 	 * @return The type of the field at the given position.
 	 */
+	@Experimental
 	public abstract <X> TypeInformation<X> getTypeAt(int pos);
 
+	@Experimental
 	protected abstract TypeComparatorBuilder<T> createTypeComparatorBuilder();
 	
 	/**
@@ -97,6 +107,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * to create the actual comparators
 	 * @return The comparator
 	 */
+	@Experimental
 	public TypeComparator<T> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) {
 
 		TypeComparatorBuilder<T> builder = createTypeComparatorBuilder();
@@ -158,6 +169,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 
 	// --------------------------------------------------------------------------------------------
 
+	@Experimental
 	protected interface TypeComparatorBuilder<T> {
 		void initializeTypeComparatorBuilder(int size);
 
@@ -166,6 +178,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 		TypeComparator<T> createTypeComparator(ExecutionConfig config);
 	}
 
+	@Experimental
 	public static class FlatFieldDescriptor {
 		private int keyPosition;
 		private TypeInformation<?> type;
@@ -196,11 +209,13 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	/**
 	 * Returns true when this type has a composite field with the given name.
 	 */
+	@Experimental
 	public boolean hasField(String fieldName) {
 		return getFieldIndex(fieldName) >= 0;
 	}
 
 	@Override
+	@Experimental
 	public boolean isKeyType() {
 		for(int i=0;i<this.getArity();i++) {
 			if (!this.getTypeAt(i).isKeyType()) {
@@ -211,6 +226,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	}
 
 	@Override
+	@Experimental
 	public boolean isSortKeyType() {
 		for(int i=0;i<this.getArity();i++) {
 			if (!this.getTypeAt(i).isSortKeyType()) {
@@ -224,6 +240,7 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * Returns the names of the composite fields of this type. The order of the returned array must
 	 * be consistent with the internal field index ordering.
 	 */
+	@Experimental
 	public abstract String[] getFieldNames();
 
 	/**
@@ -235,16 +252,20 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
 	 * This is used when translating a DataSet or DataStream to an Expression Table, when
 	 * initially renaming the fields of the underlying type.
 	 */
+	@Experimental
 	public boolean hasDeterministicFieldOrder() {
 		return false;
 	}
+
 	/**
 	 * Returns the field index of the composite field of the given name.
 	 *
 	 * @return The field index or -1 if this type does not have a field of the given name.
 	 */
+	@Experimental
 	public abstract int getFieldIndex(String fieldName);
 
+	@Experimental
 	public static class InvalidFieldReferenceException extends IllegalArgumentException {
 
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java b/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java
index 3d06c59..d96f078 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.functions;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.functions.Function;
 
 import java.io.Serializable;
@@ -31,6 +32,7 @@ import java.io.Serializable;
  * @param <IN> Type of objects to extract the key from.
  * @param <KEY> Type of key.
  */
+@Public
 public interface KeySelector<IN, KEY> extends Function, Serializable {
 	
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
index 71b1f61..c23bcc9 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.types.NullFieldException;
 
 
@@ -33,6 +34,7 @@ import org.apache.flink.types.NullFieldException;
  * Tuples are in principle serializable. However, they may contain non-serializable fields,
  * in which case serialization will fail.
  */
+@Public
 public abstract class Tuple implements java.io.Serializable {
 	
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple0.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple0.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple0.java
index c3386dc..69f6a1e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple0.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple0.java
@@ -17,6 +17,8 @@
  */
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
+
 import java.io.ObjectStreamException;
 
 /**
@@ -27,6 +29,7 @@ import java.io.ObjectStreamException;
  * 
  * @see Tuple
  */
+@Public
 public class Tuple0 extends Tuple {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple1.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple1.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple1.java
index 01c5501..9c81070 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple1.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple1.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -39,6 +40,7 @@ import org.apache.flink.util.StringUtils;
  *
  * @param <T0> The type of field 0
  */
+@Public
 public class Tuple1<T0> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple10.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple10.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple10.java
index ea24f40..5f8c062 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple10.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple10.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -48,6 +49,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T8> The type of field 8
  * @param <T9> The type of field 9
  */
+@Public
 public class Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple11.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple11.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple11.java
index b58ae6c..6f36627 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple11.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple11.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -49,6 +50,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T9> The type of field 9
  * @param <T10> The type of field 10
  */
+@Public
 public class Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple12.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple12.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple12.java
index 4e7defb..9fb8d78 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple12.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple12.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -50,6 +51,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T10> The type of field 10
  * @param <T11> The type of field 11
  */
+@Public
 public class Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple13.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple13.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple13.java
index 9543912..b8ca790 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple13.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple13.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -51,6 +52,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T11> The type of field 11
  * @param <T12> The type of field 12
  */
+@Public
 public class Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple14.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple14.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple14.java
index 36d508e..64b9fe3 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple14.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple14.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -52,6 +53,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T12> The type of field 12
  * @param <T13> The type of field 13
  */
+@Public
 public class Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple15.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple15.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple15.java
index 0d45352..9ce3b25 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple15.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple15.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -53,6 +54,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T13> The type of field 13
  * @param <T14> The type of field 14
  */
+@Public
 public class Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple16.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple16.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple16.java
index 5c57ee7..b8ba1a6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple16.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple16.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -54,6 +55,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T14> The type of field 14
  * @param <T15> The type of field 15
  */
+@Public
 public class Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple17.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple17.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple17.java
index 70da5bb..20d852b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple17.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple17.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -55,6 +56,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T15> The type of field 15
  * @param <T16> The type of field 16
  */
+@Public
 public class Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple18.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple18.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple18.java
index c221ee6..5885894 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple18.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple18.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -56,6 +57,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T16> The type of field 16
  * @param <T17> The type of field 17
  */
+@Public
 public class Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple19.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple19.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple19.java
index 3d4d262..21f36c0 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple19.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple19.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -57,6 +58,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T17> The type of field 17
  * @param <T18> The type of field 18
  */
+@Public
 public class Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple2.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple2.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple2.java
index 4ec930c..dc65761 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple2.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple2.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -40,6 +41,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T0> The type of field 0
  * @param <T1> The type of field 1
  */
+@Public
 public class Tuple2<T0, T1> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java
index 8116121..7046520 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple20.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -58,6 +59,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T18> The type of field 18
  * @param <T19> The type of field 19
  */
+@Public
 public class Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple21.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple21.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple21.java
index 6a1aeab..a826a52 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple21.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple21.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -59,6 +60,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T19> The type of field 19
  * @param <T20> The type of field 20
  */
+@Public
 public class Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple22.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple22.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple22.java
index 5f7194b..0502511 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple22.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple22.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -60,6 +61,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T20> The type of field 20
  * @param <T21> The type of field 21
  */
+@Public
 public class Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple23.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple23.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple23.java
index 35c71ce..23c036e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple23.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple23.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -61,6 +62,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T21> The type of field 21
  * @param <T22> The type of field 22
  */
+@Public
 public class Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple24.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple24.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple24.java
index 5c7f91f..d18d1d4 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple24.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple24.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -62,6 +63,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T22> The type of field 22
  * @param <T23> The type of field 23
  */
+@Public
 public class Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple25.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple25.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple25.java
index 901b838..8174824 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple25.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple25.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -63,6 +64,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T23> The type of field 23
  * @param <T24> The type of field 24
  */
+@Public
 public class Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple3.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple3.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple3.java
index 3a46222..27e19ef 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple3.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple3.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -41,6 +42,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T1> The type of field 1
  * @param <T2> The type of field 2
  */
+@Public
 public class Tuple3<T0, T1, T2> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple4.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple4.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple4.java
index 0556ad4..61e4076 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple4.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple4.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -42,6 +43,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T2> The type of field 2
  * @param <T3> The type of field 3
  */
+@Public
 public class Tuple4<T0, T1, T2, T3> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple5.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple5.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple5.java
index 15d3ea9..259db12 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple5.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple5.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -43,6 +44,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T3> The type of field 3
  * @param <T4> The type of field 4
  */
+@Public
 public class Tuple5<T0, T1, T2, T3, T4> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple6.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple6.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple6.java
index 1d45a26..f0dc9d9 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple6.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple6.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -44,6 +45,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T4> The type of field 4
  * @param <T5> The type of field 5
  */
+@Public
 public class Tuple6<T0, T1, T2, T3, T4, T5> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple7.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple7.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple7.java
index 2008117..9e0b8da 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple7.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple7.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -45,6 +46,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T5> The type of field 5
  * @param <T6> The type of field 6
  */
+@Public
 public class Tuple7<T0, T1, T2, T3, T4, T5, T6> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple8.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple8.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple8.java
index 7aaa265..2807509 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple8.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple8.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -46,6 +47,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T6> The type of field 6
  * @param <T7> The type of field 7
  */
+@Public
 public class Tuple8<T0, T1, T2, T3, T4, T5, T6, T7> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple9.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple9.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple9.java
index 86ea998..70bbdf7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple9.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/Tuple9.java
@@ -25,6 +25,7 @@
 
 package org.apache.flink.api.java.tuple;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -47,6 +48,7 @@ import org.apache.flink.util.StringUtils;
  * @param <T7> The type of field 7
  * @param <T8> The type of field 8
  */
+@Public
 public class Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8> extends Tuple {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java
index 5facb7e..4cc2965 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java
@@ -28,8 +28,10 @@ package org.apache.flink.api.java.tuple.builder;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.java.tuple.Tuple0;
 
+@Public
 public class Tuple0Builder {
 
 	private List<Tuple0> tuples = new ArrayList<Tuple0>();

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple10Builder.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple10Builder.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple10Builder.java
index a5aec21..8f5b58b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple10Builder.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple10Builder.java
@@ -28,8 +28,10 @@ package org.apache.flink.api.java.tuple.builder;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.java.tuple.Tuple10;
 
+@Public
 public class Tuple10Builder<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> {
 
 	private List<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> tuples = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/b54499b1/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple11Builder.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple11Builder.java b/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple11Builder.java
index cbbba7d..96c16a8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple11Builder.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple11Builder.java
@@ -28,8 +28,10 @@ package org.apache.flink.api.java.tuple.builder;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.flink.annotation.Public;
 import org.apache.flink.api.java.tuple.Tuple11;
 
+@Public
 public class Tuple11Builder<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> {
 
 	private List<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> tuples = new ArrayList<>();


Mime
View raw message