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

[2/3] flink git commit: [FLINK-1911] [streaming] Projection without specifying types

[FLINK-1911] [streaming] Projection without specifying types


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

Branch: refs/heads/master
Commit: 8c46155a09cd2198518fffde6bf1c8ab1a671120
Parents: ca82b0c
Author: szape <ne...@gmail.com>
Authored: Mon Apr 20 16:53:46 2015 +0200
Committer: mbalassi <mb...@apache.org>
Committed: Wed Apr 29 15:25:39 2015 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    |    5 +-
 .../api/datastream/StreamProjection.java        | 1418 +++---------------
 .../streaming/api/operators/ProjectTest.java    |   14 +-
 .../api/operators/ProjectWithoutClassTest.java  |   22 +
 4 files changed, 249 insertions(+), 1210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8c46155a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 7a6a0fa..bd5e1ca 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -616,10 +616,11 @@ public class DataStream<OUT> {
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public StreamProjection<OUT> project(int... fieldIndexes) {
-		return new StreamProjection<OUT>(this.copy(), fieldIndexes);
+	public <R extends Tuple> SingleOutputStreamOperator<R, ?> project(int... fieldIndexes) {
+		return new StreamProjection<OUT>(this.copy(), fieldIndexes).projectTupleX();
 	}
 
+
 	/**
 	 * Initiates a temporal Cross transformation.<br/>
 	 * A Cross transformation combines the elements of two {@link DataStream}s

http://git-wip-us.apache.org/repos/asf/flink/blob/8c46155a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
index 0243344..c19dbb2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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.
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.api.datastream;
 
+import com.google.common.base.Preconditions;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple1;
@@ -47,1425 +48,438 @@ import org.apache.flink.api.java.tuple.Tuple9;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.streaming.api.operators.StreamProject;
 
+import java.util.Arrays;
+
 public class StreamProjection<IN> {
 
 	private DataStream<IN> dataStream;
 	private int[] fieldIndexes;
-	private TypeInformation<IN> inTypeInfo;
 
 	protected StreamProjection(DataStream<IN> dataStream, int[] fieldIndexes) {
+		if (!dataStream.getType().isTupleType()) {
+			throw new RuntimeException("Only Tuple DataStreams can be projected");
+		}
+		if(fieldIndexes.length == 0) {
+			throw new IllegalArgumentException("project() needs to select at least one (1) field.");
+		} else if(fieldIndexes.length > Tuple.MAX_ARITY - 1) {
+			throw new IllegalArgumentException(
+					"project() may select only up to (" + (Tuple.MAX_ARITY - 1) + ") fields.");
+		}
+
+		int maxFieldIndex = (dataStream.getType()).getArity();
+		for(int i = 0; i < fieldIndexes.length; i++) {
+			Preconditions.checkElementIndex(fieldIndexes[i], maxFieldIndex);
+		}
+
 		this.dataStream = dataStream;
 		this.fieldIndexes = fieldIndexes;
-		this.inTypeInfo = dataStream.getType();
-		if (!inTypeInfo.isTupleType()) {
-			throw new RuntimeException("Only Tuple DataStreams can be projected");
+	}
+
+	/**
+	 * Chooses a projectTupleX according to the length of
+	 * {@link org.apache.flink.streaming.api.datastream.StreamProjection#fieldIndexes}
+	 *
+	 * @return The projected DataStream.
+	 * @see org.apache.flink.api.java.operators.ProjectOperator.Projection
+	 */
+	@SuppressWarnings("unchecked")
+	public <OUT extends Tuple> SingleOutputStreamOperator<OUT, ?> projectTupleX() {
+		SingleOutputStreamOperator<OUT, ?> projOperator = null;
+
+		switch (fieldIndexes.length) {
+			case 1: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple1(); break;
+			case 2: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple2(); break;
+			case 3: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple3(); break;
+			case 4: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple4(); break;
+			case 5: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple5(); break;
+			case 6: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple6(); break;
+			case 7: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple7(); break;
+			case 8: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple8(); break;
+			case 9: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple9(); break;
+			case 10: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple10(); break;
+			case 11: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple11(); break;
+			case 12: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple12(); break;
+			case 13: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple13(); break;
+			case 14: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple14(); break;
+			case 15: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple15(); break;
+			case 16: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple16(); break;
+			case 17: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple17(); break;
+			case 18: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple18(); break;
+			case 19: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple19(); break;
+			case 20: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple20(); break;
+			case 21: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple21(); break;
+			case 22: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple22(); break;
+			case 23: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple23(); break;
+			case 24: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple24(); break;
+			case 25: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple25(); break;
+			default:
+				throw new IllegalStateException("Excessive arity in tuple.");
 		}
+
+		return projOperator;
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0> SingleOutputStreamOperator<Tuple1<T0>, ?> types(Class<T0> type0) {
-		Class<?>[] types = { type0 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0> SingleOutputStreamOperator<Tuple1<T0>, ?> projectTuple1() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple1<T0>> tType = new TupleTypeInfo<Tuple1<T0>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple1<T0>> outType = (TypeInformation<Tuple1<T0>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType, new StreamProject<IN, Tuple1<T0>>(
-				fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple1<T0>>(
+				fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1> SingleOutputStreamOperator<Tuple2<T0, T1>, ?> types(Class<T0> type0,
-			Class<T1> type1) {
-		Class<?>[] types = { type0, type1 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1> SingleOutputStreamOperator<Tuple2<T0, T1>, ?> projectTuple2() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple2<T0, T1>> tType = new TupleTypeInfo<Tuple2<T0, T1>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple2<T0, T1>> outType = (TypeInformation<Tuple2<T0, T1>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple2<T0, T1>>(fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple2<T0, T1>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2> SingleOutputStreamOperator<Tuple3<T0, T1, T2>, ?> types(Class<T0> type0,
-			Class<T1> type1, Class<T2> type2) {
-		Class<?>[] types = { type0, type1, type2 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2> SingleOutputStreamOperator<Tuple3<T0, T1, T2>, ?> projectTuple3() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple3<T0, T1, T2>> tType = new TupleTypeInfo<Tuple3<T0, T1, T2>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple3<T0, T1, T2>> outType = (TypeInformation<Tuple3<T0, T1, T2>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple3<T0, T1, T2>>(fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple3<T0, T1, T2>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3> SingleOutputStreamOperator<Tuple4<T0, T1, T2, T3>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3) {
-		Class<?>[] types = { type0, type1, type2, type3 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3> SingleOutputStreamOperator<Tuple4<T0, T1, T2, T3>, ?> projectTuple4() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple4<T0, T1, T2, T3>> tType = new TupleTypeInfo<Tuple4<T0, T1, T2, T3>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple4<T0, T1, T2, T3>> outType = (TypeInformation<Tuple4<T0, T1, T2, T3>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple4<T0, T1, T2, T3>>(fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple4<T0, T1, T2, T3>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4> SingleOutputStreamOperator<Tuple5<T0, T1, T2, T3, T4>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4) {
-		Class<?>[] types = { type0, type1, type2, type3, type4 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple5<T0, T1, T2, T3, T4>> outType = (TypeInformation<Tuple5<T0, T1, T2, T3, T4>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple5<T0, T1, T2, T3, T4>>(fieldIndexes, outType));
+	public <T0, T1, T2, T3, T4> SingleOutputStreamOperator<Tuple5<T0, T1, T2, T3, T4>, ?> projectTuple5() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>> tType = new TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple5<T0, T1, T2, T3, T4>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5> SingleOutputStreamOperator<Tuple6<T0, T1, T2, T3, T4, T5>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5> SingleOutputStreamOperator<Tuple6<T0, T1, T2, T3, T4, T5>, ?> projectTuple6() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>> tType = new TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple6<T0, T1, T2, T3, T4, T5>> outType = (TypeInformation<Tuple6<T0, T1, T2, T3, T4, T5>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple6<T0, T1, T2, T3, T4, T5>>(fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple6<T0, T1, T2, T3, T4, T5>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6> SingleOutputStreamOperator<Tuple7<T0, T1, T2, T3, T4, T5, T6>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6> SingleOutputStreamOperator<Tuple7<T0, T1, T2, T3, T4, T5, T6>, ?> projectTuple7() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>> tType = new TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple7<T0, T1, T2, T3, T4, T5, T6>> outType = (TypeInformation<Tuple7<T0, T1, T2, T3, T4, T5, T6>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform("Projection", outType,
-						new StreamProject<IN, Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fieldIndexes,
-								outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7> SingleOutputStreamOperator<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7> SingleOutputStreamOperator<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>, ?> projectTuple8() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> tType = new TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> outType = (TypeInformation<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fieldIndexes,
-						outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8> SingleOutputStreamOperator<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8> SingleOutputStreamOperator<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>, ?> projectTuple9() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> tType = new TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> outType = (TypeInformation<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fieldIndexes,
-						outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> SingleOutputStreamOperator<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> SingleOutputStreamOperator<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>, ?> projectTuple10() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> tType = new TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> outType = (TypeInformation<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(
-						fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> SingleOutputStreamOperator<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> SingleOutputStreamOperator<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>, ?> projectTuple11() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> tType = new TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> outType = (TypeInformation<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream.transform("Projection", outType,
-				new StreamProject<IN, Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(
-						fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> SingleOutputStreamOperator<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> SingleOutputStreamOperator<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>, ?> projectTuple12() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> tType = new TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> outType = (TypeInformation<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> SingleOutputStreamOperator<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> SingleOutputStreamOperator<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>, ?> projectTuple13() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> tType = new TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> outType = (TypeInformation<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> SingleOutputStreamOperator<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> SingleOutputStreamOperator<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>, ?> projectTuple14() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> tType = new TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> outType = (TypeInformation<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> SingleOutputStreamOperator<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> SingleOutputStreamOperator<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>, ?> projectTuple15() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> tType = new TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> outType = (TypeInformation<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> SingleOutputStreamOperator<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> SingleOutputStreamOperator<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>, ?> projectTuple16() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> tType = new TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> outType = (TypeInformation<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> SingleOutputStreamOperator<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> SingleOutputStreamOperator<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>, ?> projectTuple17() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> tType = new TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> outType = (TypeInformation<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> SingleOutputStreamOperator<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> SingleOutputStreamOperator<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>, ?> projectTuple18() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> tType = new TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> outType = (TypeInformation<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> SingleOutputStreamOperator<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> SingleOutputStreamOperator<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>, ?> projectTuple19() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> tType = new TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> outType = (TypeInformation<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
-	 * @param type19
-	 *            The class of field '19' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> SingleOutputStreamOperator<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18, Class<T19> type19) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> SingleOutputStreamOperator<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>, ?> projectTuple20() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> tType = new TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> outType = (TypeInformation<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
-	 * @param type19
-	 *            The class of field '19' of the result Tuples.
-	 * @param type20
-	 *            The class of field '20' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> SingleOutputStreamOperator<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18, Class<T19> type19, Class<T20> type20) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
-				type20 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> SingleOutputStreamOperator<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>, ?> projectTuple21() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> tType = new TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> outType = (TypeInformation<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
-	 * @param type19
-	 *            The class of field '19' of the result Tuples.
-	 * @param type20
-	 *            The class of field '20' of the result Tuples.
-	 * @param type21
-	 *            The class of field '21' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> SingleOutputStreamOperator<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
-				type20, type21 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> SingleOutputStreamOperator<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>, ?> projectTuple22() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> tType = new TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> outType = (TypeInformation<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
-	 * @param type19
-	 *            The class of field '19' of the result Tuples.
-	 * @param type20
-	 *            The class of field '20' of the result Tuples.
-	 * @param type21
-	 *            The class of field '21' of the result Tuples.
-	 * @param type22
-	 *            The class of field '22' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> SingleOutputStreamOperator<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21,
-			Class<T22> type22) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
-				type20, type21, type22 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> SingleOutputStreamOperator<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>, ?> projectTuple23() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> tType = new TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> outType = (TypeInformation<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
-	 * @param type19
-	 *            The class of field '19' of the result Tuples.
-	 * @param type20
-	 *            The class of field '20' of the result Tuples.
-	 * @param type21
-	 *            The class of field '21' of the result Tuples.
-	 * @param type22
-	 *            The class of field '22' of the result Tuples.
-	 * @param type23
-	 *            The class of field '23' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> SingleOutputStreamOperator<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>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21,
-			Class<T22> type22, Class<T23> type23) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
-				type20, type21, type22, type23 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> SingleOutputStreamOperator<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>, ?> projectTuple24() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<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>> tType = new TupleTypeInfo<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>>(fTypes);
 
-		@SuppressWarnings("unchecked")
-		TypeInformation<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>> outType = (TypeInformation<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>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
-						outType,
-						new StreamProject<IN, 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>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, 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>>(fieldIndexes, tType));
 	}
 
 	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
-	 * fields. Requires the classes of the fields of the resulting Tuples.
-	 * 
-	 * @param type0
-	 *            The class of field '0' of the result Tuples.
-	 * @param type1
-	 *            The class of field '1' of the result Tuples.
-	 * @param type2
-	 *            The class of field '2' of the result Tuples.
-	 * @param type3
-	 *            The class of field '3' of the result Tuples.
-	 * @param type4
-	 *            The class of field '4' of the result Tuples.
-	 * @param type5
-	 *            The class of field '5' of the result Tuples.
-	 * @param type6
-	 *            The class of field '6' of the result Tuples.
-	 * @param type7
-	 *            The class of field '7' of the result Tuples.
-	 * @param type8
-	 *            The class of field '8' of the result Tuples.
-	 * @param type9
-	 *            The class of field '9' of the result Tuples.
-	 * @param type10
-	 *            The class of field '10' of the result Tuples.
-	 * @param type11
-	 *            The class of field '11' of the result Tuples.
-	 * @param type12
-	 *            The class of field '12' of the result Tuples.
-	 * @param type13
-	 *            The class of field '13' of the result Tuples.
-	 * @param type14
-	 *            The class of field '14' of the result Tuples.
-	 * @param type15
-	 *            The class of field '15' of the result Tuples.
-	 * @param type16
-	 *            The class of field '16' of the result Tuples.
-	 * @param type17
-	 *            The class of field '17' of the result Tuples.
-	 * @param type18
-	 *            The class of field '18' of the result Tuples.
-	 * @param type19
-	 *            The class of field '19' of the result Tuples.
-	 * @param type20
-	 *            The class of field '20' of the result Tuples.
-	 * @param type21
-	 *            The class of field '21' of the result Tuples.
-	 * @param type22
-	 *            The class of field '22' of the result Tuples.
-	 * @param type23
-	 *            The class of field '23' of the result Tuples.
-	 * @param type24
-	 *            The class of field '24' of the result Tuples.
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
 	 * @return The projected DataStream.
-	 * 
 	 * @see Tuple
 	 * @see DataStream
 	 */
-	public <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> SingleOutputStreamOperator<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>, ?> types(
-			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
-			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
-			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
-			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
-			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21,
-			Class<T22> type22, Class<T23> type23, Class<T24> type24) {
-		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
-				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
-				type20, type21, type22, type23, type24 };
-		if (types.length != this.fieldIndexes.length) {
-			throw new IllegalArgumentException(
-					"Numbers of projected fields and types do not match.");
-		}
-
-		@SuppressWarnings("unchecked")
-		TypeInformation<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>> outType = (TypeInformation<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>>) extractFieldTypes(
-				fieldIndexes, types, inTypeInfo);
-		return dataStream
-				.transform(
-						"Projection",
+	public <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> SingleOutputStreamOperator<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>, ?> projectTuple25() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<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>> tType = new TupleTypeInfo<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>>(fTypes);
 
-						outType,
-						new StreamProject<IN, 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>>(
-								fieldIndexes, outType));
+		return dataStream.transform("Projection", tType, new StreamProject<IN, 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>>(fieldIndexes, tType));
 	}
 
-	public static TypeInformation<?> extractFieldTypes(int[] fields, Class<?>[] givenTypes,
-			TypeInformation<?> inType) {
+	public static TypeInformation<?>[] extractFieldTypes(int[] fields, TypeInformation<?> inType) {
 
 		TupleTypeInfo<?> inTupleType = (TupleTypeInfo<?>) inType;
 		TypeInformation<?>[] fieldTypes = new TypeInformation[fields.length];
 
 		for (int i = 0; i < fields.length; i++) {
-
-			if (inTupleType.getTypeAt(fields[i]).getTypeClass() != givenTypes[i]) {
-				throw new IllegalArgumentException(
-						"Given types do not match types of input data set.");
-			}
-
 			fieldTypes[i] = inTupleType.getTypeAt(fields[i]);
 		}
 
-		return new TupleTypeInfo<Tuple>(fieldTypes);
+		return fieldTypes;
 	}
 
 }