You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2014/09/22 14:28:43 UTC

[01/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Repository: incubator-flink
Updated Branches:
  refs/heads/master 7f946cee6 -> ec562ceb5


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala
new file mode 100644
index 0000000..c702077
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+class TupleComparatorISD2Test extends TupleComparatorTestBase[(Int, String, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, String, Double)] = {
+    val ti = createTypeInformation[(Int, String, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, String, Double)]]
+      .createComparator(Array(0, 1), Array(ascending, ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, String, Double)] = {
+    val ti = createTypeInformation[(Int, String, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, String, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, "hello", 20.0),
+    (4, "world", 23.2),
+    (5, "hello", 20.0),
+    (5, "world", 20.0),
+    (6, "hello", 23.2),
+    (6, "world", 20.0),
+    (7, "hello", 20.0),
+    (7, "world", 23.2)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala
new file mode 100644
index 0000000..3850ec1
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+class TupleComparatorISD3Test extends TupleComparatorTestBase[(Int, String, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, String, Double)] = {
+    val ti = createTypeInformation[(Int, String, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, String, Double)]]
+      .createComparator(Array(0, 1, 2), Array(ascending, ascending, ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, String, Double)] = {
+    val ti = createTypeInformation[(Int, String, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, String, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, "hello", 20.0),
+    (4, "hello", 23.2),
+    (4, "world", 20.0),
+    (5, "hello", 20.0),
+    (5, "hello", 23.2),
+    (5, "world", 20.0),
+    (6, "hello", 20.0),
+    (6, "hello", 23.2)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
new file mode 100644
index 0000000..ae6f15d
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeSerializerTest._
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.StringUtils
+import org.junit.Assert
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+import scala.collection.JavaConverters._
+
+import java.util.Random
+
+class TupleSerializerTest {
+
+  @Test
+  def testTuple1Int(): Unit = {
+    val testTuples =
+      Array(Tuple1(42), Tuple1(1), Tuple1(0), Tuple1(-1), Tuple1(Int.MaxValue), Tuple1(Int.MinValue))
+    runTests(testTuples)
+  }
+
+  @Test
+  def testTuple1String(): Unit = {
+    val rnd: Random = new Random(68761564135413L)
+    val testTuples = Array(
+      Tuple1(StringUtils.getRandomString(rnd, 10, 100)),
+      Tuple1("abc"),
+      Tuple1(""),
+      Tuple1(StringUtils.getRandomString(rnd, 30, 170)),
+      Tuple1(StringUtils.getRandomString(rnd, 15, 50)),
+      Tuple1(""))
+    runTests(testTuples)
+  }
+
+  @Test
+  def testTuple1StringArray(): Unit = {
+    val rnd: Random = new Random(289347567856686223L)
+
+    val arr1 = Array(
+      "abc",
+      "",
+      StringUtils.getRandomString(rnd, 10, 100),
+      StringUtils.getRandomString(rnd, 15, 50),
+      StringUtils.getRandomString(rnd, 30, 170),
+      StringUtils.getRandomString(rnd, 14, 15),
+      "")
+    val arr2 = Array(
+      "foo",
+      "",
+      StringUtils.getRandomString(rnd, 10, 100),
+      StringUtils.getRandomString(rnd, 1000, 5000),
+      StringUtils.getRandomString(rnd, 30000, 35000),
+      StringUtils.getRandomString(rnd, 100 * 1024, 105 * 1024),
+      "bar")
+    val testTuples = Array(Tuple1(arr1), Tuple1(arr2))
+    runTests(testTuples)
+  }
+
+  @Test
+  def testTuple2StringDouble(): Unit = {
+    val rnd: Random = new Random(807346528946L)
+
+    val testTuples = Array(
+      (StringUtils.getRandomString(rnd, 10, 100), rnd.nextDouble),
+      (StringUtils.getRandomString(rnd, 10, 100), rnd.nextDouble),
+      (StringUtils.getRandomString(rnd, 10, 100), rnd.nextDouble),
+      ("", rnd.nextDouble),
+      (StringUtils.getRandomString(rnd, 10, 100), rnd.nextDouble),
+      (StringUtils.getRandomString(rnd, 10, 100), rnd.nextDouble))
+    runTests(testTuples)
+  }
+
+  @Test
+  def testTuple2StringStringArray(): Unit = {
+    val rnd: Random = new Random(289347567856686223L)
+
+    val arr1 = Array(
+      "abc",
+      "",
+      StringUtils.getRandomString(rnd, 10, 100),
+      StringUtils.getRandomString(rnd, 15, 50),
+      StringUtils.getRandomString(rnd, 30, 170),
+      StringUtils.getRandomString(rnd, 14, 15), "")
+    val arr2 = Array(
+      "foo",
+      "",
+      StringUtils.getRandomString(rnd, 10, 100),
+      StringUtils.getRandomString(rnd, 1000, 5000),
+      StringUtils.getRandomString(rnd, 30000, 35000),
+      StringUtils.getRandomString(rnd, 100 * 1024, 105 * 1024),
+      "bar")
+    val testTuples = Array(
+      (StringUtils.getRandomString(rnd, 30, 170), arr1),
+      (StringUtils.getRandomString(rnd, 30, 170), arr2),
+      (StringUtils.getRandomString(rnd, 30, 170), arr1),
+      (StringUtils.getRandomString(rnd, 30, 170), arr2),
+      (StringUtils.getRandomString(rnd, 30, 170), arr2))
+    runTests(testTuples)
+  }
+
+  @Test
+  def testTuple5CustomObjects(): Unit = {
+    val rnd: Random = new Random(807346528946L)
+
+    val a = new SimpleTypes
+    val b =  new SimpleTypes(rnd.nextInt, rnd.nextLong, rnd.nextInt.asInstanceOf[Byte],
+        StringUtils.getRandomString(rnd, 10, 100), rnd.nextInt.asInstanceOf[Short], rnd.nextDouble)
+    val c = new SimpleTypes(rnd.nextInt, rnd.nextLong, rnd.nextInt.asInstanceOf[Byte],
+        StringUtils.getRandomString(rnd, 10, 100), rnd.nextInt.asInstanceOf[Short], rnd.nextDouble)
+    val d = new SimpleTypes(rnd.nextInt, rnd.nextLong, rnd.nextInt.asInstanceOf[Byte],
+        StringUtils.getRandomString(rnd, 10, 100), rnd.nextInt.asInstanceOf[Short], rnd.nextDouble)
+    val e = new SimpleTypes(rnd.nextInt, rnd.nextLong, rnd.nextInt.asInstanceOf[Byte],
+        StringUtils.getRandomString(rnd, 10, 100), rnd.nextInt.asInstanceOf[Short], rnd.nextDouble)
+    val f = new SimpleTypes(rnd.nextInt, rnd.nextLong, rnd.nextInt.asInstanceOf[Byte],
+        StringUtils.getRandomString(rnd, 10, 100), rnd.nextInt.asInstanceOf[Short], rnd.nextDouble)
+    val g = new SimpleTypes(rnd.nextInt, rnd.nextLong, rnd.nextInt.asInstanceOf[Byte],
+        StringUtils.getRandomString(rnd, 10, 100), rnd.nextInt.asInstanceOf[Short], rnd.nextDouble)
+
+    val o1 = new ComplexNestedObject1(5626435)
+    val o2 = new ComplexNestedObject1(76923)
+    val o3 = new ComplexNestedObject1(-1100)
+    val o4 = new ComplexNestedObject1(0)
+    val o5 = new ComplexNestedObject1(44)
+
+    val co1 = new ComplexNestedObject2(rnd)
+    val co2 = new ComplexNestedObject2
+    val co3 = new ComplexNestedObject2(rnd)
+    val co4 = new ComplexNestedObject2(rnd)
+
+    val b1 = new Book(976243875L, "The Serialization Odysse", 42)
+    val b2 = new Book(0L, "Debugging byte streams", 1337)
+    val b3 = new Book(-1L, "Low level interfaces", 0xC0FFEE)
+    val b4 = new Book(Long.MaxValue, "The joy of bits and bytes", 0xDEADBEEF)
+    val b5 = new Book(Long.MaxValue, "Winnign a prize for creative test strings", 0xBADF00)
+    val b6 = new Book(-2L, "Distributed Systems", 0xABCDEF0123456789L)
+
+    val list = List("A", "B", "C", "D", "E")
+    val ba1 = new BookAuthor(976243875L, list.asJava, "Arno Nym")
+
+    val list2 = List[String]()
+    val ba2 = new BookAuthor(987654321L, list2.asJava, "The Saurus")
+
+    val testTuples = Array(
+      (a, b1, o1, ba1, co1),
+      (b, b2, o2, ba2, co2),
+      (c, b3, o3, ba1, co3),
+      (d, b2, o4, ba1, co4),
+      (e, b4, o5, ba2, co4),
+      (f, b5, o1, ba2, co4),
+      (g, b6, o4, ba1, co2))
+    runTests(testTuples)
+  }
+
+  private final def runTests[T <: Product : TypeInformation](instances: Array[T]) {
+    try {
+      val tupleTypeInfo = implicitly[TypeInformation[T]].asInstanceOf[TupleTypeInfoBase[T]]
+      val serializer = tupleTypeInfo.createSerializer
+      val tupleClass = tupleTypeInfo.getTypeClass
+      val test = new TupleSerializerTestInstance[T](serializer, tupleClass, -1, instances)
+      test.testAll()
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        Assert.fail(e.getMessage)
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala
new file mode 100644
index 0000000..3e6e1d7
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.junit.Assert._
+import org.apache.flink.api.common.typeutils.SerializerTestInstance
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.junit.Assert
+import org.junit.Test
+
+
+class TupleSerializerTestInstance[T <: Product] (
+    serializer: TypeSerializer[T],
+    typeClass: Class[T],
+    length: Int,
+    testData: Array[T])
+  extends SerializerTestInstance[T](serializer, typeClass, length, testData: _*) {
+
+  @Test
+  override def testInstantiate(): Unit = {
+    try {
+      val serializer: TypeSerializer[T] = getSerializer
+      val instance: T = serializer.createInstance
+      assertNotNull("The created instance must not be null.", instance)
+      val tpe: Class[T] = getTypeClass
+      assertNotNull("The test is corrupt: type class is null.", tpe)
+      // We cannot check this because Tuple1 instances are not actually of type Tuple1
+      // but something like Tuple1$mcI$sp
+//      assertEquals("Type of the instantiated object is wrong.", tpe, instance.getClass)
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Exception in test: " + e.getMessage)
+      }
+    }
+  }
+
+  protected override def deepEquals(message: String, shouldTuple: T, isTuple: T) {
+    Assert.assertEquals(shouldTuple.productArity, isTuple.productArity)
+    for (i <- 0 until shouldTuple.productArity) {
+      val should = shouldTuple.productElement(i)
+      val is = isTuple.productElement(i)
+      if (should.getClass.isArray) {
+        should match {
+          case booleans: Array[Boolean] =>
+            Assert.assertTrue(message, booleans.sameElements(is.asInstanceOf[Array[Boolean]]))
+          case bytes: Array[Byte] =>
+            assertArrayEquals(message, bytes, is.asInstanceOf[Array[Byte]])
+          case shorts: Array[Short] =>
+            assertArrayEquals(message, shorts, is.asInstanceOf[Array[Short]])
+          case ints: Array[Int] =>
+            assertArrayEquals(message, ints, is.asInstanceOf[Array[Int]])
+          case longs: Array[Long] =>
+            assertArrayEquals(message, longs, is.asInstanceOf[Array[Long]])
+          case floats: Array[Float] =>
+            assertArrayEquals(message, floats, is.asInstanceOf[Array[Float]], 0.0f)
+          case doubles: Array[Double] =>
+            assertArrayEquals(message, doubles, is.asInstanceOf[Array[Double]], 0.0)
+          case chars: Array[Char] =>
+            assertArrayEquals(message, chars, is.asInstanceOf[Array[Char]])
+          case _ =>
+            assertArrayEquals(
+              message,
+              should.asInstanceOf[Array[AnyRef]],
+              is.asInstanceOf[Array[AnyRef]])
+        }
+      } else {
+        assertEquals(message, should, is)
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala
new file mode 100644
index 0000000..5722664
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/PairComparatorTestBase.scala
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime.tuple.base
+
+import org.junit.Assert.assertTrue
+import org.junit.Assert.fail
+import org.apache.flink.api.common.typeutils.TypePairComparator
+import org.junit.Test
+
+/**
+ * Abstract test base for PairComparators.
+ */
+abstract class PairComparatorTestBase[T, R] {
+  protected def createComparator(ascending: Boolean): TypePairComparator[T, R]
+
+  protected def getSortedTestData: (Array[T], Array[R])
+
+  @Test
+  def testEqualityWithReference(): Unit = {
+    try {
+      val comparator = getComparator(true)
+      
+      val (dataT, dataR) = getSortedData
+      for (i <- 0 until dataT.length) {
+        comparator.setReference(dataT(i))
+        assertTrue(comparator.equalToReference(dataR(i)))
+      }
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Exception in test: " + e.getMessage)
+      }
+    }
+  }
+
+  @Test
+  def testInequalityWithReference(): Unit = {
+    testGreatSmallAscDescWithReference(true)
+    testGreatSmallAscDescWithReference(false)
+  }
+
+  protected def testGreatSmallAscDescWithReference(ascending: Boolean) {
+    try {
+      val (dataT, dataR) = getSortedData
+      val comparator = getComparator(ascending)
+      for (x <- 0 until (dataT.length - 1)) {
+        for (y <- (x + 1) until dataR.length) {
+          comparator.setReference(dataT(x))
+          if (ascending) {
+            assertTrue(comparator.compareToReference(dataR(y)) > 0)
+          }
+          else {
+            assertTrue(comparator.compareToReference(dataR(y)) < 0)
+          }
+        }
+      }
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Exception in test: " + e.getMessage)
+      }
+    }
+  }
+
+  protected def getComparator(ascending: Boolean): TypePairComparator[T, R] = {
+    val comparator: TypePairComparator[T, R] = createComparator(ascending)
+    if (comparator == null) {
+      throw new RuntimeException("Test case corrupt. Returns null as comparator.")
+    }
+    comparator
+  }
+
+  protected def getSortedData: (Array[T], Array[R]) = {
+    val (dataT, dataR) = getSortedTestData
+
+    if (dataT == null || dataR == null) {
+      throw new RuntimeException("Test case corrupt. Returns null as test data.")
+    }
+    if (dataT.length < 2 || dataR.length < 2) {
+      throw new RuntimeException("Test case does not provide enough sorted test data.")
+    }
+    (dataT, dataR)
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
new file mode 100644
index 0000000..5370e61
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime.tuple.base
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase
+import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleComparator}
+import org.junit.Assert._
+
+abstract class TupleComparatorTestBase[T <: Product] extends ComparatorTestBase[T] {
+  protected override def deepEquals(message: String, should: T, is: T) {
+    for (i <- 0 until should.productArity) {
+      assertEquals(should.productElement(i), is.productElement(i))
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
new file mode 100644
index 0000000..bbdb1a6
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
@@ -0,0 +1,258 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.types
+
+import java.io.DataInput
+import java.io.DataOutput
+import org.apache.flink.api.java.typeutils._
+import org.apache.flink.types.{IntValue, StringValue}
+import org.apache.hadoop.io.Writable
+import org.junit.Assert
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+class MyWritable extends Writable {
+  def write(out: DataOutput) {
+  }
+
+  def readFields(in: DataInput) {
+  }
+}
+
+case class CustomCaseClass(a: String, b: Int)
+
+class CustomType(var myField1: String, var myField2: Int) {
+  def this() {
+    this(null, 0)
+  }
+}
+
+class MyObject[A](var a: A)
+
+class TypeInformationGenTest {
+
+  @Test
+  def testBasicType(): Unit = {
+    val ti = createTypeInformation[Boolean]
+
+    Assert.assertTrue(ti.isBasicType)
+    Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, ti)
+    Assert.assertEquals(classOf[java.lang.Boolean], ti.getTypeClass)
+  }
+
+  @Test
+  def testWritableType(): Unit = {
+    val ti = createTypeInformation[MyWritable]
+
+    Assert.assertTrue(ti.isInstanceOf[WritableTypeInfo[_]])
+    Assert.assertEquals(classOf[MyWritable], ti.asInstanceOf[WritableTypeInfo[_]].getTypeClass)
+  }
+
+  @Test
+  def testTupleWithBasicTypes(): Unit = {
+    val ti = createTypeInformation[(Int, Long, Double, Float, Boolean, String, Char, Short, Byte)]
+
+    Assert.assertTrue(ti.isTupleType)
+    Assert.assertEquals(9, ti.getArity)
+    Assert.assertTrue(ti.isInstanceOf[TupleTypeInfoBase[_]])
+    val tti = ti.asInstanceOf[TupleTypeInfoBase[_]]
+    Assert.assertEquals(classOf[Tuple9[_,_,_,_,_,_,_,_,_]], tti.getTypeClass)
+    for (i <- 0 until 0) {
+      Assert.assertTrue(tti.getTypeAt(i).isInstanceOf[BasicTypeInfo[_]])
+    }
+
+    Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, tti.getTypeAt(0))
+    Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, tti.getTypeAt(1))
+    Assert.assertEquals(BasicTypeInfo.DOUBLE_TYPE_INFO, tti.getTypeAt(2))
+    Assert.assertEquals(BasicTypeInfo.FLOAT_TYPE_INFO, tti.getTypeAt(3))
+    Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, tti.getTypeAt(4))
+    Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, tti.getTypeAt(5))
+    Assert.assertEquals(BasicTypeInfo.CHAR_TYPE_INFO, tti.getTypeAt(6))
+    Assert.assertEquals(BasicTypeInfo.SHORT_TYPE_INFO, tti.getTypeAt(7))
+    Assert.assertEquals(BasicTypeInfo.BYTE_TYPE_INFO, tti.getTypeAt(8))
+  }
+
+  @Test
+  def testTupleWithTuples(): Unit = {
+    val ti = createTypeInformation[(Tuple1[String], Tuple1[Int], Tuple2[Long, Long])]
+
+    Assert.assertTrue(ti.isTupleType())
+    Assert.assertEquals(3, ti.getArity)
+    Assert.assertTrue(ti.isInstanceOf[TupleTypeInfoBase[_]])
+    val tti = ti.asInstanceOf[TupleTypeInfoBase[_]]
+    Assert.assertEquals(classOf[Tuple3[_, _, _]], tti.getTypeClass)
+    Assert.assertTrue(tti.getTypeAt(0).isTupleType())
+    Assert.assertTrue(tti.getTypeAt(1).isTupleType())
+    Assert.assertTrue(tti.getTypeAt(2).isTupleType())
+    Assert.assertEquals(classOf[Tuple1[_]], tti.getTypeAt(0).getTypeClass)
+    Assert.assertEquals(classOf[Tuple1[_]], tti.getTypeAt(1).getTypeClass)
+    Assert.assertEquals(classOf[Tuple2[_, _]], tti.getTypeAt(2).getTypeClass)
+    Assert.assertEquals(1, tti.getTypeAt(0).getArity)
+    Assert.assertEquals(1, tti.getTypeAt(1).getArity)
+    Assert.assertEquals(2, tti.getTypeAt(2).getArity)
+    Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO,
+      tti.getTypeAt(0).asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(0))
+    Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO,
+      tti.getTypeAt(1).asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(0))
+    Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO,
+      tti.getTypeAt(2).asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(0))
+    Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO,
+      tti.getTypeAt(2).asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(1))
+  }
+
+  @Test
+  def testCaseClass(): Unit = {
+    val ti = createTypeInformation[CustomCaseClass]
+
+    Assert.assertTrue(ti.isTupleType)
+    Assert.assertEquals(2, ti.getArity)
+    Assert.assertEquals(
+      BasicTypeInfo.STRING_TYPE_INFO,
+      ti.asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(0))
+    Assert.assertEquals(
+      BasicTypeInfo.INT_TYPE_INFO,
+      ti.asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(1))
+    Assert.assertEquals(
+      classOf[CustomCaseClass],ti.asInstanceOf[TupleTypeInfoBase[_]].getTypeClass())
+  }
+
+  @Test
+  def testCustomType(): Unit = {
+    val ti = createTypeInformation[CustomType]
+
+    Assert.assertFalse(ti.isBasicType)
+    Assert.assertFalse(ti.isTupleType)
+    Assert.assertTrue(ti.isInstanceOf[GenericTypeInfo[_]])
+    Assert.assertEquals(ti.getTypeClass, classOf[CustomType])
+  }
+
+  @Test
+  def testTupleWithCustomType(): Unit = {
+    val ti = createTypeInformation[(Long, CustomType)]
+
+    Assert.assertTrue(ti.isTupleType)
+    Assert.assertEquals(2, ti.getArity)
+    val tti = ti.asInstanceOf[TupleTypeInfoBase[_]]
+    Assert.assertEquals(classOf[Tuple2[_, _]], tti.getTypeClass)
+    Assert.assertEquals(classOf[java.lang.Long], tti.getTypeAt(0).getTypeClass)
+    Assert.assertTrue(tti.getTypeAt(1).isInstanceOf[GenericTypeInfo[_]])
+    Assert.assertEquals(classOf[CustomType], tti.getTypeAt(1).getTypeClass)
+  }
+
+  @Test
+  def testValue(): Unit = {
+    val ti = createTypeInformation[StringValue]
+
+    Assert.assertFalse(ti.isBasicType)
+    Assert.assertFalse(ti.isTupleType)
+    Assert.assertTrue(ti.isInstanceOf[ValueTypeInfo[_]])
+    Assert.assertEquals(ti.getTypeClass, classOf[StringValue])
+    Assert.assertTrue(TypeExtractor.getForClass(classOf[StringValue])
+      .isInstanceOf[ValueTypeInfo[_]])
+    Assert.assertEquals(TypeExtractor.getForClass(classOf[StringValue]).getTypeClass,
+      ti.getTypeClass)
+  }
+
+  @Test
+  def testTupleOfValues(): Unit = {
+    val ti = createTypeInformation[(StringValue, IntValue)]
+    Assert.assertFalse(ti.isBasicType)
+    Assert.assertTrue(ti.isTupleType)
+    Assert.assertEquals(
+      classOf[StringValue],
+      ti.asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(0).getTypeClass)
+    Assert.assertEquals(
+      classOf[IntValue],
+      ti.asInstanceOf[TupleTypeInfoBase[_]].getTypeAt(1).getTypeClass)
+  }
+
+
+  @Test
+  def testBasicArray(): Unit = {
+    val ti = createTypeInformation[Array[String]]
+
+    Assert.assertFalse(ti.isBasicType)
+    Assert.assertFalse(ti.isTupleType)
+    Assert.assertTrue(ti.isInstanceOf[BasicArrayTypeInfo[_, _]] ||
+      ti.isInstanceOf[ObjectArrayTypeInfo[_, _]])
+    if (ti.isInstanceOf[BasicArrayTypeInfo[_, _]]) {
+      Assert.assertEquals(BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO, ti)
+    }
+    else {
+      Assert.assertEquals(
+        BasicTypeInfo.STRING_TYPE_INFO,
+        ti.asInstanceOf[ObjectArrayTypeInfo[_, _]].getComponentInfo)
+    }
+  }
+
+  @Test
+  def testPrimitiveArray(): Unit = {
+    val ti = createTypeInformation[Array[Boolean]]
+
+    Assert.assertTrue(ti.isInstanceOf[PrimitiveArrayTypeInfo[_]])
+    Assert.assertEquals(ti, PrimitiveArrayTypeInfo.BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO)
+  }
+
+  @Test
+  def testCustomArray(): Unit = {
+    val ti = createTypeInformation[Array[CustomType]]
+    Assert.assertTrue(ti.isInstanceOf[ObjectArrayTypeInfo[_, _]])
+    Assert.assertEquals(
+      classOf[CustomType],
+      ti.asInstanceOf[ObjectArrayTypeInfo[_, _]].getComponentType)
+  }
+
+  @Test
+  def testTupleArray(): Unit = {
+    val ti = createTypeInformation[Array[(String, String)]]
+
+    Assert.assertTrue(ti.isInstanceOf[ObjectArrayTypeInfo[_, _]])
+    val oati = ti.asInstanceOf[ObjectArrayTypeInfo[_, _]]
+    Assert.assertTrue(oati.getComponentInfo.isTupleType)
+    val tti = oati.getComponentInfo.asInstanceOf[TupleTypeInfoBase[_]]
+    Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, tti.getTypeAt(0))
+    Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, tti.getTypeAt(1))
+  }
+
+  @Test
+  def testParamertizedCustomObject(): Unit = {
+    val ti = createTypeInformation[MyObject[String]]
+
+    Assert.assertTrue(ti.isInstanceOf[GenericTypeInfo[_]])
+  }
+
+  @Test
+  def testTupleWithPrimitiveArray(): Unit = {
+    val ti = createTypeInformation[(Array[Int], Array[Double], Array[Long],
+      Array[Byte], Array[Char], Array[Float], Array[Short], Array[Boolean],
+      Array[String])]
+
+    val tti = ti.asInstanceOf[TupleTypeInfoBase[_]]
+    Assert.assertEquals(PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(0))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(1))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.LONG_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(2))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(3))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.CHAR_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(4))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.FLOAT_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(5))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.SHORT_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(6))
+    Assert.assertEquals(PrimitiveArrayTypeInfo.BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO, tti.getTypeAt(7))
+    Assert.assertEquals(BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO, tti.getTypeAt(8))
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
index 4892e07..2426bc9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
@@ -1,39 +1,39 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.examples.scala.graph.ComputeEdgeDegrees;
-
-public class ComputeEdgeDegreesITCase extends org.apache.flink.test.recordJobTests.ComputeEdgeDegreesITCase {
-
-	public ComputeEdgeDegreesITCase(Configuration config) {
-		super(config);
-	}
-	
-	@Override
-	protected Plan getTestJob() {
-		ComputeEdgeDegrees computeDegrees = new ComputeEdgeDegrees();
-		return computeDegrees.getScalaPlan(
-				config.getInteger("ComputeEdgeDegreesTest#NumSubtasks", DOP),
-				edgesPath, resultPath);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.configuration.Configuration;
+//import org.apache.flink.examples.scala.graph.ComputeEdgeDegrees;
+//
+//public class ComputeEdgeDegreesITCase extends org.apache.flink.test.recordJobTests.ComputeEdgeDegreesITCase {
+//
+//	public ComputeEdgeDegreesITCase(Configuration config) {
+//		super(config);
+//	}
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		ComputeEdgeDegrees computeDegrees = new ComputeEdgeDegrees();
+//		return computeDegrees.getScalaPlan(
+//				config.getInteger("ComputeEdgeDegreesTest#NumSubtasks", DOP),
+//				edgesPath, resultPath);
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
index 14cbc42..69a5c9a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
@@ -1,34 +1,34 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.examples.scala.graph.ConnectedComponents;
-
-public class ConnectedComponentsITCase extends org.apache.flink.test.iterative.ConnectedComponentsITCase {
-
-	@Override
-	protected Plan getTestJob() {
-		ConnectedComponents cc = new ConnectedComponents();
-		Plan plan = cc.getScalaPlan(verticesPath, edgesPath, resultPath, 100);
-		plan.setDefaultParallelism(DOP);
-		return plan;
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.examples.scala.graph.ConnectedComponents;
+//
+//public class ConnectedComponentsITCase extends org.apache.flink.test.iterative.ConnectedComponentsITCase {
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		ConnectedComponents cc = new ConnectedComponents();
+//		Plan plan = cc.getScalaPlan(verticesPath, edgesPath, resultPath, 100);
+//		plan.setDefaultParallelism(DOP);
+//		return plan;
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
index de0e0ca..ab8c563 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
@@ -1,39 +1,39 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.examples.scala.graph.EnumTrianglesOnEdgesWithDegrees;
-
-public class EnumTrianglesOnEdgesWithDegreesITCase extends org.apache.flink.test.recordJobTests.EnumTrianglesOnEdgesWithDegreesITCase {
-
-	public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) {
-		super(config);
-	}
-	
-	@Override
-	protected Plan getTestJob() {
-		EnumTrianglesOnEdgesWithDegrees enumTriangles = new EnumTrianglesOnEdgesWithDegrees();
-		return enumTriangles.getScalaPlan(
-				config.getInteger("EnumTrianglesTest#NumSubtasks", DOP),
-				edgesPath, resultPath);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.configuration.Configuration;
+//import org.apache.flink.examples.scala.graph.EnumTrianglesOnEdgesWithDegrees;
+//
+//public class EnumTrianglesOnEdgesWithDegreesITCase extends org.apache.flink.test.recordJobTests.EnumTrianglesOnEdgesWithDegreesITCase {
+//
+//	public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) {
+//		super(config);
+//	}
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		EnumTrianglesOnEdgesWithDegrees enumTriangles = new EnumTrianglesOnEdgesWithDegrees();
+//		return enumTriangles.getScalaPlan(
+//				config.getInteger("EnumTrianglesTest#NumSubtasks", DOP),
+//				edgesPath, resultPath);
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/IterativeKMeansITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/IterativeKMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/IterativeKMeansITCase.java
deleted file mode 100644
index 7e5aa30..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/IterativeKMeansITCase.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import java.util.Locale;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.examples.scala.datamining.KMeans;
-import org.apache.flink.test.testdata.KMeansData;
-import org.apache.flink.test.util.RecordAPITestBase;
-
-public class IterativeKMeansITCase extends RecordAPITestBase {
-
-	static {
-		Locale.setDefault(Locale.US);
-	}
-	
-	protected String pointsPath;
-	protected String clusterPath;
-	protected String resultPath;
-	
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		pointsPath = createTempFile("datapoints.txt", KMeansData.DATAPOINTS);
-		clusterPath = createTempFile("initial_centers.txt", KMeansData.INITIAL_CENTERS);
-		resultPath = getTempDirPath("resulting_centers");
-	}
-	
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(KMeansData.CENTERS_AFTER_20_ITERATIONS_SINGLE_DIGIT, resultPath);
-	}
-	
-
-	@Override
-	protected Plan getTestJob() {
-		KMeans kmi = new KMeans();
-		return kmi.getScalaPlan(4, pointsPath, clusterPath, resultPath, 20);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
index afa1c69..ca5a707 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
@@ -1,49 +1,49 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.examples.scala.relational.RelationalQuery;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Locale;
-
-@RunWith(Parameterized.class)
-public class RelationalQueryITCase extends org.apache.flink.test.recordJobTests.TPCHQuery3ITCase {
-
-	public RelationalQueryITCase(Configuration config) {
-		super(config);
-		Locale.setDefault(Locale.US);
-	}
-
-	@Override
-	protected Plan getTestJob()  {
-
-		RelationalQuery tpch3 = new RelationalQuery();
-		return tpch3.getScalaPlan(
-				config.getInteger("dop", 1),
-				ordersPath,
-				lineitemsPath,
-				resultPath,
-				'F', 1993, "5");
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.configuration.Configuration;
+//import org.apache.flink.examples.scala.relational.RelationalQuery;
+//import org.junit.runner.RunWith;
+//import org.junit.runners.Parameterized;
+//
+//import java.util.Locale;
+//
+//@RunWith(Parameterized.class)
+//public class RelationalQueryITCase extends org.apache.flink.test.recordJobTests.TPCHQuery3ITCase {
+//
+//	public RelationalQueryITCase(Configuration config) {
+//		super(config);
+//		Locale.setDefault(Locale.US);
+//	}
+//
+//	@Override
+//	protected Plan getTestJob()  {
+//
+//		RelationalQuery tpch3 = new RelationalQuery();
+//		return tpch3.getScalaPlan(
+//				config.getInteger("dop", 1),
+//				ordersPath,
+//				lineitemsPath,
+//				resultPath,
+//				'F', 1993, "5");
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
index 6b2106a..4d0eb24 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
@@ -1,54 +1,54 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.examples.scala.graph.TransitiveClosureNaive;
-import org.apache.flink.test.util.RecordAPITestBase;
-
-public class TransitiveClosureNaiveITCase extends RecordAPITestBase {
-
-	protected String verticesPath = null;
-	protected String edgesPath = null;
-	protected String resultPath = null;
-
-	private static final String VERTICES = "0\n1\n2";
-	private static final String EDGES = "0|1\n1|2";
-	private static final String EXPECTED = "0|0|0\n0|1|1\n0|2|2\n1|1|0\n1|2|1\n2|2|0";
-
-	@Override
-	protected void preSubmit() throws Exception {
-		verticesPath = createTempFile("vertices.txt", VERTICES);
-		edgesPath = createTempFile("edges.txt", EDGES);
-		resultPath = getTempDirPath("transitiveClosure");
-	}
-
-	@Override
-	protected Plan getTestJob() {
-		TransitiveClosureNaive transitiveClosureNaive = new TransitiveClosureNaive();
-		// "2" is the number of iterations here
-		return transitiveClosureNaive.getScalaPlan(DOP, 2, verticesPath, edgesPath, resultPath);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(EXPECTED, resultPath);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.examples.scala.graph.TransitiveClosureNaive;
+//import org.apache.flink.test.util.RecordAPITestBase;
+//
+//public class TransitiveClosureNaiveITCase extends RecordAPITestBase {
+//
+//	protected String verticesPath = null;
+//	protected String edgesPath = null;
+//	protected String resultPath = null;
+//
+//	private static final String VERTICES = "0\n1\n2";
+//	private static final String EDGES = "0|1\n1|2";
+//	private static final String EXPECTED = "0|0|0\n0|1|1\n0|2|2\n1|1|0\n1|2|1\n2|2|0";
+//
+//	@Override
+//	protected void preSubmit() throws Exception {
+//		verticesPath = createTempFile("vertices.txt", VERTICES);
+//		edgesPath = createTempFile("edges.txt", EDGES);
+//		resultPath = getTempDirPath("transitiveClosure");
+//	}
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		TransitiveClosureNaive transitiveClosureNaive = new TransitiveClosureNaive();
+//		// "2" is the number of iterations here
+//		return transitiveClosureNaive.getScalaPlan(DOP, 2, verticesPath, edgesPath, resultPath);
+//	}
+//
+//	@Override
+//	protected void postSubmit() throws Exception {
+//		compareResultsByLinesInMemory(EXPECTED, resultPath);
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
index 5cc5223..63c598c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
@@ -1,32 +1,32 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.examples.scala.relational.WebLogAnalysis;
-
-public class WebLogAnalysisITCase extends org.apache.flink.test.recordJobTests.WebLogAnalysisITCase {
-
-	@Override
-	protected Plan getTestJob() {
-		WebLogAnalysis webLogAnalysis = new WebLogAnalysis();
-		return webLogAnalysis.getScalaPlan(DOP, docsPath, ranksPath, visitsPath, resultPath);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.examples.scala.relational.WebLogAnalysis;
+//
+//public class WebLogAnalysisITCase extends org.apache.flink.test.recordJobTests.WebLogAnalysisITCase {
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		WebLogAnalysis webLogAnalysis = new WebLogAnalysis();
+//		return webLogAnalysis.getScalaPlan(DOP, docsPath, ranksPath, visitsPath, resultPath);
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
index e15ec0e..c0e1db0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
@@ -16,17 +16,36 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.exampleScalaPrograms;
 
-import org.apache.flink.api.common.Plan;
 import org.apache.flink.examples.scala.wordcount.WordCount;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class WordCountITCase extends JavaProgramTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	public WordCountITCase(){
+		setDegreeOfParallelism(4);
+		setNumTaskTracker(2);
+		setTaskManagerNumSlots(2);
+	}
 
-public class WordCountITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
 
 	@Override
-	protected Plan getTestJob() {
-		WordCount wc = new WordCount();
-		return wc.getScalaPlan(DOP, textPath, resultPath);
+	protected void testProgram() throws Exception {
+		WordCount.main(new String[] { textPath, resultPath });
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
index 702a87a..94ec224 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
@@ -1,33 +1,33 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.examples.scala.wordcount.WordCountWithUserDefinedType;
-
-
-public class WordCountPactValueITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
-
-	@Override
-	protected Plan getTestJob() {
-		WordCountWithUserDefinedType wc = new WordCountWithUserDefinedType();
-		return wc.getScalaPlan(DOP, textPath, resultPath);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.examples.scala.wordcount.WordCountWithUserDefinedType;
+//
+//
+//public class WordCountPactValueITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		WordCountWithUserDefinedType wc = new WordCountWithUserDefinedType();
+//		return wc.getScalaPlan(DOP, textPath, resultPath);
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
index f4703da..5f53f72 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
@@ -1,31 +1,31 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.examples.scala.wordcount.WordCountWithCount;
-
-public class WordCountWithCountFunctionITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
-
-	@Override
-	protected Plan getTestJob() {
-		return new WordCountWithCount().getScalaPlan(DOP, textPath, resultPath);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.exampleScalaPrograms;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.examples.scala.wordcount.WordCountWithCount;
+//
+//public class WordCountWithCountFunctionITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		return new WordCountWithCount().getScalaPlan(DOP, textPath, resultPath);
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
index 4b32df8..814bca0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
@@ -1,92 +1,92 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.iterative;
-
-import java.util.Collection;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.examples.scala.graph.LineRank;
-import org.apache.flink.test.util.RecordAPITestBase;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public class LineRankITCase extends RecordAPITestBase {
-	
-	private static final String SOURCE_INCIDENCE = "1,1,1\n" +
-	                                               "2,1,1\n" +
-	                                               "3,1,1\n" +
-	                                               "4,2,1\n" +
-	                                               "5,3,1\n" +
-	                                               "6,3,1\n" +
-	                                               "7,4,1\n" +
-	                                               "8,4,1\n" +
-	                                               "9,5,1\n";
-	
-	private static final String TARGET_INCIDENCE = "1,2,1\n" +
-	                                               "2,3,1\n" +
-	                                               "3,4,1\n" +
-	                                               "4,3,1\n" +
-	                                               "5,2,1\n" +
-	                                               "6,5,1\n" +
-	                                               "7,1,1\n" +
-	                                               "8,3,1\n" +
-	                                               "9,4,1\n";
-
-	protected String sourcesPath;
-	protected String targetsPath;
-	protected String resultPath;
-	
-	
-	public LineRankITCase(Configuration config) {
-		super(config);
-		setTaskManagerNumSlots(DOP);
-	}
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		sourcesPath = createTempFile("sourceIncidence.txt", SOURCE_INCIDENCE);
-		targetsPath = createTempFile("targetIncidence.txt", TARGET_INCIDENCE);
-		resultPath = getTempFilePath("results");
-	}
-
-	@Override
-	protected Plan getTestJob() {
-		LineRank lr = new LineRank();
-		
-		Plan plan = lr.getScalaPlan(
-			config.getInteger("NumSubtasks", 1),
-			sourcesPath,
-			targetsPath,
-			9,
-			resultPath);
-		return plan;
-	}
-
-	@Parameters
-	public static Collection<Object[]> getConfigurations() {
-		Configuration config1 = new Configuration();
-		config1.setInteger("NumSubtasks", DOP);
-		config1.setInteger("NumIterations", 5);
-		return toParameterList(config1);
-	}
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.test.iterative;
+//
+//import java.util.Collection;
+//
+//import org.apache.flink.api.common.Plan;
+//import org.apache.flink.configuration.Configuration;
+//import org.apache.flink.examples.scala.graph.LineRank;
+//import org.apache.flink.test.util.RecordAPITestBase;
+//import org.junit.runner.RunWith;
+//import org.junit.runners.Parameterized;
+//import org.junit.runners.Parameterized.Parameters;
+//
+//@RunWith(Parameterized.class)
+//public class LineRankITCase extends RecordAPITestBase {
+//
+//	private static final String SOURCE_INCIDENCE = "1,1,1\n" +
+//	                                               "2,1,1\n" +
+//	                                               "3,1,1\n" +
+//	                                               "4,2,1\n" +
+//	                                               "5,3,1\n" +
+//	                                               "6,3,1\n" +
+//	                                               "7,4,1\n" +
+//	                                               "8,4,1\n" +
+//	                                               "9,5,1\n";
+//
+//	private static final String TARGET_INCIDENCE = "1,2,1\n" +
+//	                                               "2,3,1\n" +
+//	                                               "3,4,1\n" +
+//	                                               "4,3,1\n" +
+//	                                               "5,2,1\n" +
+//	                                               "6,5,1\n" +
+//	                                               "7,1,1\n" +
+//	                                               "8,3,1\n" +
+//	                                               "9,4,1\n";
+//
+//	protected String sourcesPath;
+//	protected String targetsPath;
+//	protected String resultPath;
+//
+//
+//	public LineRankITCase(Configuration config) {
+//		super(config);
+//		setTaskManagerNumSlots(DOP);
+//	}
+//
+//	@Override
+//	protected void preSubmit() throws Exception {
+//		sourcesPath = createTempFile("sourceIncidence.txt", SOURCE_INCIDENCE);
+//		targetsPath = createTempFile("targetIncidence.txt", TARGET_INCIDENCE);
+//		resultPath = getTempFilePath("results");
+//	}
+//
+//	@Override
+//	protected Plan getTestJob() {
+//		LineRank lr = new LineRank();
+//
+//		Plan plan = lr.getScalaPlan(
+//			config.getInteger("NumSubtasks", 1),
+//			sourcesPath,
+//			targetsPath,
+//			9,
+//			resultPath);
+//		return plan;
+//	}
+//
+//	@Parameters
+//	public static Collection<Object[]> getConfigurations() {
+//		Configuration config1 = new Configuration();
+//		config1.setInteger("NumSubtasks", DOP);
+//		config1.setInteger("NumIterations", 5);
+//		return toParameterList(config1);
+//	}
+//}


[37/60] git commit: Add SemanticPropertiesTranslationTest for Scala API

Posted by al...@apache.org.
Add SemanticPropertiesTranslationTest for Scala API

Fix join operator to user proper EquiJoin constructor so that semantic
properties are actually retrieved.

Fix typo, getSematicProperties -> getSemanticProperties


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

Branch: refs/heads/master
Commit: 1a4147f31856d5d21c02281d8077eada86a3a861
Parents: 83debdb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 12 15:50:09 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../spargel/java/SpargelTranslationTest.java    |   8 +-
 .../api/java/operators/OperatorTranslation.java |   4 +-
 .../java/operators/SingleInputUdfOperator.java  |   6 +-
 .../api/java/operators/TwoInputUdfOperator.java |   6 +-
 .../flink/api/java/operators/UdfOperator.java   |   2 +-
 .../apache/flink/api/scala/joinDataSet.scala    |   2 +-
 .../SemanticPropertiesTranslationTest.scala     | 195 +++++++++++++++++++
 7 files changed, 209 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java b/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
index 155aa3f..09643cf 100644
--- a/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
+++ b/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
@@ -100,8 +100,8 @@ public class SpargelTranslationTest {
 			
 			// validate that the semantic properties are set as they should
 			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
-			assertTrue(solutionSetJoin.getSematicProperties().getForwardedField1(0).contains(0));
-			assertTrue(solutionSetJoin.getSematicProperties().getForwardedField2(0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardedField1(0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardedField2(0).contains(0));
 			
 			TwoInputUdfOperator<?, ?, ?, ?> edgesJoin = (TwoInputUdfOperator<?, ?, ?, ?>) solutionSetJoin.getInput1();
 			
@@ -179,8 +179,8 @@ public class SpargelTranslationTest {
 			
 			// validate that the semantic properties are set as they should
 			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
-			assertTrue(solutionSetJoin.getSematicProperties().getForwardedField1(0).contains(0));
-			assertTrue(solutionSetJoin.getSematicProperties().getForwardedField2(0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardedField1(0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardedField2(0).contains(0));
 			
 			TwoInputUdfOperator<?, ?, ?, ?> edgesJoin = (TwoInputUdfOperator<?, ?, ?, ?>) solutionSetJoin.getInput1();
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index e44302a..0603993 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -128,7 +128,7 @@ public class OperatorTranslation {
 			}
 			
 			// set the semantic properties
-			dataFlowOp.setSemanticProperties(udfOp.getSematicProperties());
+			dataFlowOp.setSemanticProperties(udfOp.getSemanticProperties());
 		}
 		
 		return dataFlowOp;
@@ -160,7 +160,7 @@ public class OperatorTranslation {
 			}
 			
 			// set the semantic properties
-			dataFlowOp.setSemanticProperties(udfOp.getSematicProperties());
+			dataFlowOp.setSemanticProperties(udfOp.getSemanticProperties());
 		}
 		
 		return dataFlowOp;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
index 5f81b62..c550408 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
@@ -142,17 +142,17 @@ public abstract class SingleInputUdfOperator<IN, OUT, O extends SingleInputUdfOp
 	}
 
 	@Override
-	public SingleInputSemanticProperties getSematicProperties() {
+	public SingleInputSemanticProperties getSemanticProperties() {
 		return this.udfSemantics;
 	}
 
 	/**
 	 * Sets the semantic properties for the user-defined function (UDF). The semantic properties
 	 * define how fields of tuples and other objects are modified or preserved through this UDF.
-	 * The configured properties can be retrieved via {@link UdfOperator#getSematicProperties()}.
+	 * The configured properties can be retrieved via {@link UdfOperator#getSemanticProperties()}.
 	 *
 	 * @param properties The semantic properties for the UDF.
-	 * @see UdfOperator#getSematicProperties()
+	 * @see UdfOperator#getSemanticProperties()
 	 */
 	public void setSemanticProperties(SingleInputSemanticProperties properties) {
 		this.udfSemantics = properties;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
index 030ad3f..4237ae1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
@@ -184,17 +184,17 @@ public abstract class TwoInputUdfOperator<IN1, IN2, OUT, O extends TwoInputUdfOp
 	}
 
 	@Override
-	public DualInputSemanticProperties getSematicProperties() {
+	public DualInputSemanticProperties getSemanticProperties() {
 		return this.udfSemantics;
 	}
 
 	/**
 	 * Sets the semantic properties for the user-defined function (UDF). The semantic properties
 	 * define how fields of tuples and other objects are modified or preserved through this UDF.
-	 * The configured properties can be retrieved via {@link UdfOperator#getSematicProperties()}.
+	 * The configured properties can be retrieved via {@link UdfOperator#getSemanticProperties()}.
 	 *
 	 * @param properties The semantic properties for the UDF.
-	 * @see UdfOperator#getSematicProperties()
+	 * @see UdfOperator#getSemanticProperties()
 	 */
 	public void setSemanticProperties(DualInputSemanticProperties properties) {
 		this.udfSemantics = properties;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
index 8af3322..5f2e3ed 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
@@ -61,7 +61,7 @@ public interface UdfOperator<O extends UdfOperator<O>> {
 	 * 
 	 * @return The semantic properties of the UDF.
 	 */
-	SemanticProperties getSematicProperties();
+	SemanticProperties getSemanticProperties();
 	
 	// --------------------------------------------------------------------------------------------
 	// Fluent API methods

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index a2b09e1..3f9f6e9 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -144,7 +144,7 @@ private[flink] class JoinDataSetImpl[T, O](
     val generatedFunction: FlatJoinFunction[T, O, R] = new WrappingFlatJoinFunction[T, O, R](fun)
 
     val joinOperator = new EquiJoin[T, O, R](thisSet, otherSet, thisKeys,
-      otherKeys, generatedFunction, implicitly[TypeInformation[R]], JoinHint.OPTIMIZER_CHOOSES)
+      otherKeys, generatedFunction, fun, implicitly[TypeInformation[R]], JoinHint.OPTIMIZER_CHOOSES)
     wrap(joinOperator)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1a4147f3/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
new file mode 100644
index 0000000..0769891
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.functions
+
+import org.junit.Assert._
+import org.apache.flink.api.common.functions.RichJoinFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.operators.SingleInputSemanticProperties
+import org.apache.flink.api.common.operators.base.{JoinOperatorBase, GenericDataSinkBase,
+MapOperatorBase}
+import org.apache.flink.api.common.operators.util.FieldSet
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+/**
+ * This is a minimal test to verify that semantic annotations are evaluated against
+ * the type information properly translated correctly to the common data flow API.
+ *
+ * This covers only the constant fields annotations currently !!!
+ */
+class SemanticPropertiesTranslationTest {
+  /**
+   * A mapper that preserves all fields over a tuple data set.
+   */
+  @Test
+  def translateUnaryFunctionAnnotationTuplesWildCard(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+
+      val input = env.fromElements((3l, "test", 42))
+      input.map(new WildcardConstantMapper[(Long, String, Int)]).print()
+
+      val plan = env.createProgramPlan()
+
+      val sink: GenericDataSinkBase[_] = plan.getDataSinks.iterator.next
+
+      val mapper: MapOperatorBase[_, _, _] = sink.getInput.asInstanceOf[MapOperatorBase[_, _, _]]
+
+      val semantics: SingleInputSemanticProperties = mapper.getSemanticProperties
+      val fw1: FieldSet = semantics.getForwardedField(0)
+      val fw2: FieldSet = semantics.getForwardedField(1)
+      val fw3: FieldSet = semantics.getForwardedField(2)
+
+      assertNotNull(fw1)
+      assertNotNull(fw2)
+      assertNotNull(fw3)
+      assertTrue(fw1.contains(0))
+      assertTrue(fw2.contains(1))
+      assertTrue(fw3.contains(2))
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Exception in test: " + e.getMessage)
+      }
+    }
+  }
+
+  /**
+   * A mapper that preserves fields 0, 1, 2 of a tuple data set.
+   */
+  @Test
+  def translateUnaryFunctionAnnotationTuples(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+
+      val input = env.fromElements((3l, "test", 42))
+      input.map(new IndividualConstantMapper[Long, String, Int]).print()
+
+      val plan = env.createProgramPlan()
+
+      val sink: GenericDataSinkBase[_] = plan.getDataSinks.iterator.next
+
+      val mapper: MapOperatorBase[_, _, _] = sink.getInput.asInstanceOf[MapOperatorBase[_, _, _]]
+
+      val semantics: SingleInputSemanticProperties = mapper.getSemanticProperties
+      val fw1: FieldSet = semantics.getForwardedField(0)
+      val fw2: FieldSet = semantics.getForwardedField(1)
+      val fw3: FieldSet = semantics.getForwardedField(2)
+
+      assertNotNull(fw1)
+      assertNotNull(fw2)
+      assertNotNull(fw3)
+      assertTrue(fw1.contains(0))
+      assertTrue(fw2.contains(1))
+      assertTrue(fw3.contains(2))
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Exception in test: " + e.getMessage)
+      }
+    }
+  }
+
+  /**
+   * A join that preserves tuple fields from both sides.
+   */
+  @Test
+  def translateBinaryFunctionAnnotationTuples(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+
+      val input1 = env.fromElements((3l, "test"))
+      val input2 = env.fromElements((3l, 3.1415))
+
+      input1.join(input2).where(0).equalTo(0)(
+        new ForwardingTupleJoin[Long, String, Long, Double]).print()
+
+      val plan = env.createProgramPlan()
+      val sink: GenericDataSinkBase[_] = plan.getDataSinks.iterator.next
+
+      val join: JoinOperatorBase[_, _, _, _] =
+        sink.getInput.asInstanceOf[JoinOperatorBase[_, _, _, _]]
+
+      val semantics = join.getSemanticProperties
+      val fw11: FieldSet = semantics.getForwardedField1(0)
+      val fw12: FieldSet = semantics.getForwardedField1(1)
+      val fw21: FieldSet = semantics.getForwardedField2(0)
+      val fw22: FieldSet = semantics.getForwardedField2(1)
+
+      assertNull(fw11)
+      assertNull(fw21)
+      assertNotNull(fw12)
+      assertNotNull(fw22)
+      assertTrue(fw12.contains(0))
+      assertTrue(fw22.contains(1))
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Exception in test: " + e.getMessage)
+      }
+    }
+  }
+}
+
+
+@ConstantFields(Array("*"))
+class WildcardConstantMapper[T] extends RichMapFunction[T, T] {
+  def map(value: T): T = {
+    value
+  }
+}
+
+@ConstantFields(Array("0->0;1->1;2->2"))
+class IndividualConstantMapper[X, Y, Z] extends RichMapFunction[(X, Y, Z), (X, Y, Z)] {
+  def map(value: (X, Y, Z)): (X, Y, Z) = {
+    value
+  }
+}
+
+@ConstantFields(Array("0"))
+class ZeroConstantMapper[T] extends RichMapFunction[T, T] {
+  def map(value: T): T = {
+    value
+  }
+}
+
+@ConstantFieldsFirst(Array("1 -> 0"))
+@ConstantFieldsSecond(Array("1 -> 1"))
+class ForwardingTupleJoin[A, B, C, D] extends RichJoinFunction[(A, B),  (C, D), (B, D)] {
+  def join(first: (A, B), second: (C, D)): (B, D) = {
+    (first._2, second._2)
+  }
+}
+
+@ConstantFieldsFirst(Array("0 -> 0"))
+@ConstantFieldsSecond(Array("0 -> 1"))
+class ForwardingBasicJoin[A, B] extends RichJoinFunction[A, B, (A, B)] {
+  def join(first: A, second: B): (A, B) = {
+    (first, second)
+  }
+}
+


[48/60] [doc] Switch parser to kramdown, normalize Headings

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/internal_overview.md
----------------------------------------------------------------------
diff --git a/docs/internal_overview.md b/docs/internal_overview.md
index 4c71c6e..a220228 100644
--- a/docs/internal_overview.md
+++ b/docs/internal_overview.md
@@ -2,8 +2,6 @@
 title:  "Overview of Flink System Architecture & Internals"
 ---
 
-# Overview
-
 This documentation provides an overview of the architecture of the Flink system
 and its components. It is intended as guide to contributors, and people
 that are interested in the technology behind Flink.
@@ -13,7 +11,7 @@ We kindly ask anyone that adds and changes components to eventually provide a pa
 or pull request that updates these documents as well.*
 
 
-### Architectures and Components
+## Architectures and Components
 
 - [General Architecture and Process Model](internal_general_arch.html)
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/iterations.md
----------------------------------------------------------------------
diff --git a/docs/iterations.md b/docs/iterations.md
index 0517322..cb1a1fb 100644
--- a/docs/iterations.md
+++ b/docs/iterations.md
@@ -2,6 +2,9 @@
 title:  "Iterations"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Iterative algorithms occur in many domains of data analysis, such as *machine learning* or *graph analysis*. Such algorithms are crucial in order to realize the promise of Big Data to extract meaningful information out of your data. With increasing interest to run these kinds of algorithms on very large data sets, there is a need to execute iterations in a massively parallel fashion.
 
 Flink programs implement iterative algorithms by defining a **step function** and embedding it into a special iteration operator. There are two  variants of this operator: **Iterate** and **Delta Iterate**. Both operators repeatedly invoke the step function on the current iteration state until a certain termination condition is reached.
@@ -58,7 +61,6 @@ The following table provides an overview of both operators:
 	</tr>
 </table>
 
-<section id="iterate">
 Iterate Operator
 ----------------
 
@@ -80,7 +82,7 @@ There are multiple options to specify **termination conditions** for an iteratio
 
 You can also think about the iterate operator in pseudo-code:
 
-```java
+~~~java
 IterationState state = getInitialState();
 
 while (!terminationCriterion()) {
@@ -88,7 +90,7 @@ while (!terminationCriterion()) {
 }
 
 setFinalState(state);
-```
+~~~
 
 <div class="panel panel-default">
 	<div class="panel-body">
@@ -108,19 +110,18 @@ In the following example, we **iteratively incremenet a set numbers**:
   3. **Next Partial Solution**: The output of the step function will be the output of the map operator, i.e. records with incremented integers.
   4. **Iteration Result**: After ten iterations, the initial numbers will have been incremented ten times, resulting in integers `11` to `15`.
 
-```
+~~~
 // 1st           2nd                       10th
 map(1) -> 2      map(2) -> 3      ...      map(10) -> 11
 map(2) -> 3      map(3) -> 4      ...      map(11) -> 12
 map(3) -> 4      map(4) -> 5      ...      map(12) -> 13
 map(4) -> 5      map(5) -> 6      ...      map(13) -> 14
 map(5) -> 6      map(6) -> 7      ...      map(14) -> 15
-```
+~~~
 
 Note that **1**, **2**, and **4** can be arbitrary data flows.
-</section>
 
-<section id="delta_iterate">
+
 Delta Iterate Operator
 ----------------------
 
@@ -141,7 +142,7 @@ The default **termination condition** for delta iterations is specified by the *
 
 You can also think about the iterate operator in pseudo-code:
 
-```java
+~~~java
 IterationState workset = getInitialState();
 IterationState solution = getInitialSolution();
 
@@ -152,7 +153,7 @@ while (!terminationCriterion()) {
 }
 
 setFinalState(solution);
-```
+~~~
 
 <div class="panel panel-default">
 	<div class="panel-body">
@@ -176,9 +177,9 @@ In the lower subgraph **ID 5** (*cyan*) is the **minimum ID**. All vertices of t
 In the **2nd iteration**, the workset size has already decreased from seven to five elements (vertices 2, 3, 4, 6, and 7). These are part of the iteration and further propagate their current minimum IDs. After this iteration, the lower subgraph has already converged (**cold part** of the graph), as it has no elements in the workset, whereas the upper half needs a further iteration (**hot part** of the graph) for the two remaining workset elements (vertices 3 and 4).
 
 The iteration **terminates**, when the workset is empty after the **3rd iteration**.
-</section>
 
-<section id="supersteps">
+<a href="#supersteps"></a>
+
 Superstep Synchronization
 -------------------------
 
@@ -187,4 +188,3 @@ We referred to each execution of the step function of an iteration operator as *
 <p class="text-center">
     <img alt="Supersteps" width="50%" src="img/iterations_supersteps.png" />
 </p>
-</section>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/java_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/java_api_examples.md b/docs/java_api_examples.md
index 0fffbf9..a45b39e 100644
--- a/docs/java_api_examples.md
+++ b/docs/java_api_examples.md
@@ -2,16 +2,19 @@
 title:  "Java API Examples"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 The following example programs showcase different applications of Flink 
 from simple word counting to graph algorithms. The code samples illustrate the 
 use of [Flink's Java API](java_api_guide.html). 
 
 The full source code of the following and more examples can be found in the __flink-java-examples__ module.
 
-# Word Count
+## Word Count
 WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
 
-```java
+~~~java
 // get input data
 DataSet<String> text = getTextDataSet(env);
 
@@ -40,17 +43,17 @@ public static final class Tokenizer extends FlatMapFunction<String, Tuple2<Strin
         }
     }
 }
-```
+~~~
 
 The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java  "WordCount example" %} implements the above described algorithm with input parameters: `<text input path>, <output path>`. As test data, any text file will do.
 
-# Page Rank
+## Page Rank
 
 The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
 
 In this simple example, PageRank is implemented with a [bulk iteration](java_api_guide.html#iterations) and a fixed number of iterations.
 
-```java
+~~~java
 // get input data
 DataSet<Tuple2<Long, Double>> pagesWithRanks = getPagesWithRanksDataSet(env);
 DataSet<Tuple2<Long, Long[]>> pageLinkLists = getLinksDataSet(env);
@@ -116,7 +119,7 @@ public static final class EpsilonFilter
         return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
     }
 }
-```
+~~~
 
 The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java "PageRank program" %} implements the above example.
 It requires the following parameters to run: `<pages input path>, <links input path>, <output path>, <num pages>, <num iterations>`.
@@ -129,13 +132,13 @@ Input files are plain text files and must be formatted as follows:
 
 For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
 
-# Connected Components
+## Connected Components
 
 The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
 
 This implementation uses a [delta iteration](iterations.html): Vertices that have not changed their component ID do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
 
-```java
+~~~java
 // read vertex and edge data
 DataSet<Long> vertices = getVertexDataSet(env);
 DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
@@ -207,7 +210,7 @@ public static final class ComponentIdFilter
         }
     }
 }
-```
+~~~
 
 The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java "ConnectedComponents program" %} implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
 
@@ -217,13 +220,13 @@ Input files are plain text files and must be formatted as follows:
 - Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
     * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
 
-# Relational Query
+## Relational Query
 
 The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
 
 The example implements the following SQL query.
 
-```sql
+~~~sql
 SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
     FROM orders, lineitem
 WHERE l_orderkey = o_orderkey
@@ -231,11 +234,11 @@ WHERE l_orderkey = o_orderkey
     AND YEAR(o_orderdate) > 1993
     AND o_orderpriority LIKE "5%"
 GROUP BY l_orderkey, o_shippriority;
-```
+~~~
 
 The Flink Java program, which implements the above query looks as follows.
 
-```java
+~~~java
 // get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
 DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
 // get lineitem data set: (orderkey, extendedprice)
@@ -278,7 +281,7 @@ DataSet<Tuple3<Integer, Integer, Double>> priceSums =
 
 // emit result
 priceSums.writeAsCsv(outputPath);
-```
+~~~
 
 The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java "Relational Query program" %} implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>`.
 
@@ -288,17 +291,17 @@ Take the following steps to generate arbitrary large input files for the provide
 1.  Download and unpack DBGEN
 2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
 
-```bash
+~~~bash
 DATABASE = DB2
 MACHINE  = LINUX
 WORKLOAD = TPCH
 CC       = gcc
-```
+~~~
 
 1.  Build DBGEN using *make*
 2.  Generate lineitem and orders relations using dbgen. A scale factor
     (-s) of 1 results in a generated data set with about 1 GB size.
 
-```bash
+~~~bash
 ./dbgen -T o -s 1
-```
\ No newline at end of file
+~~~
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/java_api_guide.md
----------------------------------------------------------------------
diff --git a/docs/java_api_guide.md b/docs/java_api_guide.md
deleted file mode 100644
index 9f615af..0000000
--- a/docs/java_api_guide.md
+++ /dev/null
@@ -1,1261 +0,0 @@
----
-title: "Java API Programming Guide"
----
-
-<section id="top">
-Introduction
-------------
-
-Analysis programs in Flink are regular Java programs that implement transformations on data sets (e.g., filtering, mapping, joining, grouping). The data sets are initially created from certain sources (e.g., by reading files, or from collections). Results are returned via sinks, which may for example write the data to (distributed) files, or to standard output (for example the command line terminal). Flink programs run in a variety of contexts, standalone, or embedded in other programs. The execution can happen in a local JVM, or on clusters of many machines.
-
-In order to create your own Flink program, we encourage you to start with the [program skeleton](#skeleton) and gradually add your own [transformations](#transformations). The remaining sections act as references for additional operations and advanced features.
-
-
-<section id="toc">
-<div id="docs_05_toc">
-  <div class="list-group">
-{% for sublink in page.toc %}
-   <a href="#{{ sublink.anchor }}" class="list-group-item">{{forloop.index}}. <strong>{{ sublink.title }}</strong></a>
-{% endfor %}
-  </div>
-</div>
-
-<section id="example">
-Example Program
----------------
-
-The following program is a complete, working example of WordCount. You can copy &amp; paste the code to run it locally. You only have to include Flink's Java API library into your project (see Section [Linking with Flink](#linking)) and specify the imports. Then you are ready to go!
-
-```java
-public class WordCountExample {
-    public static void main(String[] args) throws Exception {
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-	    DataSet<String> text = env.fromElements(
-            "Who's there?",
-            "I think I hear them. Stand, ho! Who's there?");
-
-        DataSet<Tuple2<String, Integer>> wordCounts = text
-            .flatMap(new LineSplitter())
-            .groupBy(0)
-            .sum(1);
-
-        wordCounts.print();
-
-        env.execute("Word Count Example");
-    }
-
-    public static class LineSplitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
-        @Override
-        public void flatMap(String line, Collector<Tuple2<String, Integer>> out) {
-            for (String word : line.split(" ")) {
-                out.collect(new Tuple2<String, Integer>(word, 1));
-            }
-        }
-    }
-}
-```
-
-[Back to top](#top)
-
-<section id="linking">
-Linking with Flink
--------------------------
-
-To write programs with Flink, you need to include Flink’s Java API library in your project.
-
-The simplest way to do this is to use the [quickstart scripts](java_api_quickstart.html). They create a blank project from a template (a Maven Archetype), which sets up everything for you. To manually create the project, you can use the archetype and create a project by calling:
-
-```bash
-mvn archetype:generate /
-    -DarchetypeGroupId=org.apache.flink/
-    -DarchetypeArtifactId=flink-quickstart-java /
-    -DarchetypeVersion={{site.FLINK_VERSION_STABLE }}
-```
-
-If you want to add Flink to an existing Maven project, add the following entry to your *dependencies* section in the *pom.xml* file of your project:
-
-```xml
-<dependency>
-  <groupId>org.apache.flink</groupId>
-  <artifactId>flink-java</artifactId>
-  <version>{{site.FLINK_VERSION_STABLE }}</version>
-</dependency>
-<dependency>
-  <groupId>org.apache.flink</groupId>
-  <artifactId>flink-clients</artifactId>
-  <version>{{site.FLINK_VERSION_STABLE }}</version>
-</dependency>
-```
-
-If you are using Flink together with Hadoop, the version of the dependency may vary depending on the version of Hadoop (or more specifically, HDFS) that you want to use Flink with.
-Please refer to the [downloads page]({{site.baseurl}}/downloads.html) for a list of available versions, and instructions on how to link with custom versions of Hadoop.
-
-In order to link against the latest SNAPSHOT versions of the code, please follow [this guide]({{site.baseurl}}/downloads.html#nightly).
-
-The *flink-clients* dependency is only necessary to invoke the Flink program locally (for example to run it standalone for testing and debugging). 
-If you intend to only export the program as a JAR file and [run it on a cluster](cluster_execution.html), you can skip that dependency.
-
-[Back to top](#top)
-
-<section id="skeleton">
-Program Skeleton
-----------------
-
-As we already saw in the example, Flink programs look like regular Java
-programs with a `main()` method. Each program consists of the same basic parts:
-
-1. Obtain an `ExecutionEnvironment`,
-2. Load/create the initial data,
-3. Specify transformations on this data,
-4. Specify where to put the results of your computations, and
-5. Execute your program.
-
-We will now give an overview of each of those steps but please refer
-to the respective sections for more details. Note that all {% gh_link /flink-java/src/main/java/org/apache/flink/api/java "core classes of the Java API" %} are found in the package `org.apache.flink.api.java`.
-
-The `ExecutionEnvironment` is the basis for all Flink programs. You can
-obtain one using these static methods on class `ExecutionEnvironment`:
-
-```java
-getExecutionEnvironment()
-
-createLocalEnvironment()
-createLocalEnvironment(int degreeOfParallelism)
-
-createRemoteEnvironment(String host, int port, String... jarFiles)
-createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles)
-```
-
-Typically, you only need to use `getExecutionEnvironment()`, since this
-will do the right thing depending on the context: if you are executing
-your program inside an IDE or as a regular Java program it will create
-a local environment that will execute your program on your local machine. If
-you created a JAR file from you program, and invoke it through the [command line](cli.html)
-or the [web interface](web_client.html),
-the Flink cluster manager will
-execute your main method and `getExecutionEnvironment()` will return
-an execution environment for executing your program on a cluster.
-
-For specifying data sources the execution environment has several methods
-to read from files using various methods: you can just read them line by line,
-as CSV files, or using completely custom data input formats. To just read
-a text file as a sequence of lines, you can use:
-
-```java
-final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-DataSet<String> text = env.readTextFile("file:///path/to/file");
-```
-
-This will give you a `DataSet` on which you can then apply transformations. For
-more information on data sources and input formats, please refer to
-[Data Sources](#data_sources).
-
-Once you have a `DataSet` you can apply transformations to create a new
-`DataSet` which you can then write to a file, transform again, or
-combine with other `DataSet`s. You apply transformations by calling
-methods on `DataSet` with your own custom transformation function. For example,
-a map transformation looks like this:
-
-```java
-DataSet<String> input = ...;
-
-DataSet<Integer> tokenized = text.map(new MapFunction<String, Integer>() {
-    @Override
-    public Integer map(String value) {
-        return Integer.parseInt(value);
-    }
-});
-```
-
-This will create a new `DataSet` by converting every String in the original
-set to an Integer. For more information and a list of all the transformations,
-please refer to [Transformations](#transformations).
-
-Once you have a `DataSet` that needs to be written to disk you call one
-of these methods on `DataSet`:
-
-```java
-writeAsText(String path)
-writeAsCsv(String path)
-write(FileOutputFormat<T> outputFormat, String filePath)
-
-print()
-```
-
-The last method is only useful for developing/debugging on a local machine,
-it will output the contents of the `DataSet` to standard output. (Note that in
-a cluster, the result goes to the standard out stream of the cluster nodes and ends
-up in the *.out* files of the workers).
-The first two do as the name suggests, the third one can be used to specify a
-custom data output format. Keep in mind, that these calls do not actually
-write to a file yet. Only when your program is completely specified and you
-call the `execute` method on your `ExecutionEnvironment` are all the
-transformations executed and is data written to disk. Please refer
-to [Data Sinks](#data_sinks) for more information on writing to files and also
-about custom data output formats.
-
-Once you specified the complete program you need to call `execute` on
-the `ExecutionEnvironment`. This will either execute on your local
-machine or submit your program for execution on a cluster, depending on
-how you created the execution environment.
-[Back to top](#top)
-
-
-<section id="lazyeval">
-Lazy Evaluation
----------------
-
-All Flink programs are executed lazily: When the program's main method is executed, the data loading and transformations do not happen directly. Rather, each operation is created and added to the program's plan. The operations are actually executed when one of the `execute()` methods is invoked on the ExecutionEnvironment object. Whether the program is executed locally or on a cluster depends on the environment of the program.
-
-The lazy evaluation lets you construct sophisticated programs that Flink executes as one holistically planned unit.
-[Back to top](#top)
-
-<section id="transformations">
-Transformations
----------------
-
-Data transformations transform one or more DataSets into a new DataSet. Programs can combine multiple transformations into
-sophisticated assemblies.
-
-This section gives a brief overview of the available transformations. The [transformations documentation](java_api_transformations.html)
-has full description of all transformations with examples.
-
-<table class="table table-bordered">
-  <thead>
-    <tr>
-      <th class="text-center" style="width: 20%">Transformation</th>
-      <th class="text-center">Description</th>
-    </tr>
-  </thead>
-
-  <tbody>
-    <tr>
-      <td><strong>Map</strong></td>
-      <td>
-        <p>Takes one element and produces one element.</p>
-{% highlight java %}
-data.map(new MapFunction<String, Integer>() {
-  public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>FlatMap</strong></td>
-      <td>
-        <p>Takes one element and produces zero, one, or more elements. </p>
-{% highlight java %}
-data.flatMap(new FlatMapFunction<String, String>() {
-  public void flatMap(String value, Collector<String> out) {
-    for (String s : value.split(" ")) {
-      out.collect(s);
-    }
-  }
-});
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>MapPartition</strong></td>
-      <td>
-        <p>Transforms a parallel partition in a single function call. The function get the partition as an `Iterable` stream and
-           can produce an arbitrary number of result values. The number of elements in each partition depends on the degree-of-parallelism
-           and previous operations.</p>
-{% highlight java %}
-data.mapPartition(new MapPartitionFunction<String, Long>() {
-  public void mapPartition(Iterable<String> values, Collector<Long> out) {
-    long c = 0;
-    for (String s : values) {
-      c++;
-    }
-    out.collect(c);
-  }
-});
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>Filter</strong></td>
-      <td>
-        <p>Evaluates a boolean function for each element and retains those for which the function returns true.</p>
-{% highlight java %}
-data.filter(new FilterFunction<Integer>() {
-  public boolean filter(Integer value) { return value > 1000; }
-});
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>Reduce</strong></td>
-      <td>
-        <p>Combines a group of elements into a single element by repeatedly combining two elements into one.
-           Reduce may be applied on a full data set, or on a grouped data set.</p>
-{% highlight java %}
-data.reduce(new ReduceFunction<Integer> {
-  public Integer reduce(Integer a, Integer b) { return a + b; }
-});
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>ReduceGroup</strong></td>
-      <td>
-        <p>Combines a group of elements into one or more elements. ReduceGroup may be applied on a full data set, or on a grouped data set.</p>
-{% highlight java %}
-data.reduceGroup(new GroupReduceFunction<Integer, Integer> {
-  public void reduceGroup(Iterable<Integer> values, Collector<Integer> out) {
-    int prefixSum = 0;
-    for (Integer i : values) {
-      prefixSum += i;
-      out.collect(prefixSum);
-    }
-  }
-});
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>Aggregate</strong></td>
-      <td>
-        <p>Aggregates a group of values into a single value. Aggregation functions can be thought of as built-in reduce functions. Aggregate may be applied on a full data set, or on a grouped data set.</p>
-{% highlight java %}
-Dataset<Tuple3<Integer, String, Double>> input = // [...]
-DataSet<Tuple3<Integer, String, Double>> output = input.aggregate(SUM, 0).and(MIN, 2);
-{% endhighlight %}
-	<p>You can also use short-hand syntax for minimum, maximum, and sum aggregations.</p>
-	{% highlight java %}
-	Dataset<Tuple3<Integer, String, Double>> input = // [...]
-DataSet<Tuple3<Integer, String, Double>> output = input.sum(0).andMin(2);
-	{% endhighlight %}
-      </td>
-    </tr>
-
-    </tr>
-      <td><strong>Join</strong></td>
-      <td>
-        Joins two data sets by creating all pairs of elements that are equal on their keys. Optionally uses a JoinFunction to turn the pair of elements into a single element, or a FlatJoinFunction to turn the pair of elements into arbitararily many (including none) elements. See <a href="#keys">keys</a> on how to define join keys.
-{% highlight java %}
-result = input1.join(input2)
-               .where(0)       // key of the first input (tuple field 0)
-               .equalTo(1);    // key of the second input (tuple field 1)
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>CoGroup</strong></td>
-      <td>
-        <p>The two-dimensional variant of the reduce operation. Groups each input on one or more fields and then joins the groups. The transformation function is called per pair of groups. See <a href="#keys">keys</a> on how to define coGroup keys.</p>
-{% highlight java %}
-data1.coGroup(data2)
-     .where(0)
-     .equalTo(1)
-     .with(new CoGroupFunction<String, String, String>() {
-         public void coGroup(Iterable<String> in1, Iterable<String> in2, Collector<String> out) {
-           out.collect(...);
-         }
-      });
-{% endhighlight %}
-      </td>
-    </tr>
-
-    <tr>
-      <td><strong>Cross</strong></td>
-      <td>
-        <p>Builds the Cartesian product (cross product) of two inputs, creating all pairs of elements. Optionally uses a CrossFunction to turn the pair of elements into a single element</p>
-{% highlight java %}
-DataSet<Integer> data1 = // [...]
-DataSet<String> data2 = // [...]
-DataSet<Tuple2<Integer, String>> result = data1.cross(data2);
-{% endhighlight %}
-      </td>
-    </tr>
-    <tr>
-      <td><strong>Union</strong></td>
-      <td>
-        <p>Produces the union of two data sets. This operation happens implicitly if more than one data set is used for a specific function input.</p>
-{% highlight java %}
-DataSet<String> data1 = // [...]
-DataSet<String> data2 = // [...]
-DataSet<String> result = data1.union(data2);
-{% endhighlight %}
-      </td>
-    </tr>
-  </tbody>
-</table>
-
-----------
-
-The following transformations are available on data sets of Tuples:
-
-<table class="table table-bordered">
-  <thead>
-    <tr>
-      <th class="text-center" stype="width: 20%">Transformation</th>
-      <th class="text-center">Description</th>
-    </tr>
-  </thead>
-  <tbody>
-   <tr>
-      <td><strong>Project</strong></td>
-      <td>
-        <p>Selects a subset of fields from the tuples</p>
-{% highlight java %}
-DataSet<Tuple3<Integer, Double, String>> in = // [...]
-DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
-{% endhighlight %}
-      </td>
-    </tr>
-  </tbody>
-</table>
-
-The [parallelism](#parallelism) of a transformation can be defined by `setParallelism(int)`. `name(String)` assigns a custom name to a transformation which is helpful for debugging. The same is possible for [Data Sources](#data_sources) and [Data Sinks](#data_sinks). 
-
-[Back to Top](#top)
-
-<section id="keys">
-Defining Keys
--------------
-
-Some transformations (join, coGroup) require that a key is defined on
-its argument DataSets, and other transformations (Reduce, GroupReduce,
-Aggregate) allow that the DataSet is grouped on a key before they are
-applied.
-
-A DataSet is grouped as
-{% highlight java %}
-DataSet<...> input = // [...]
-DataSet<...> reduced = input
-	.groupBy(/*define key here*/)
-	.reduceGroup(/*do something*/);
-{% endhighlight %}
-
-The data model of Flink is not based on key-value pairs. Therefore,
-you do not need to physically pack the data set types into keys and
-values. Keys are "virtual": they are defined as functions over the
-actual data to guide the grouping operator.
-
-The simplest case is grouping a data set of Tuples on one or more
-fields of the Tuple:
-{% highlight java %}
-DataSet<Tuple3<Integer,String,Long>> input = // [...]
-DataSet<Tuple3<Integer,String,Long> grouped = input
-	.groupBy(0)
-	.reduceGroup(/*do something*/);
-{% endhighlight %}
-
-The data set is grouped on the first field of the tuples (the one of
-Integer type). The GroupReduceFunction will thus receive groups with
-the same value of the first field.
-
-{% highlight java %}
-DataSet<Tuple3<Integer,String,Long>> input = // [...]
-DataSet<Tuple3<Integer,String,Long> grouped = input
-	.groupBy(0,1)
-	.reduce(/*do something*/);
-{% endhighlight %}
-
-The data set is grouped on the composite key consisting of the first and the
-second fields, therefore the GroupReduceFuntion will receive groups
-with the same value for both fields.
-
-In general, key definition is done via a "key selector" function, which
-takes as argument one dataset element and returns a key of an
-arbitrary data type by performing an arbitrary computation on this
-element. For example:
-{% highlight java %}
-// some ordinary POJO
-public class WC {public String word; public int count;}
-DataSet<WC> words = // [...]
-DataSet<WC> wordCounts = words
-                         .groupBy(
-                           new KeySelector<WC, String>() {
-                             public String getKey(WC wc) { return wc.word; }
-                           })
-                         .reduce(/*do something*/);
-{% endhighlight %}
-
-Remember that keys are not only used for grouping, but also joining and matching data sets:
-{% highlight java %}
-// some POJO
-public class Rating {
-  public String name;
-  public String category;
-  public int points;
-}
-DataSet<Rating> ratings = // [...]
-DataSet<Tuple2<String, Double>> weights = // [...]
-DataSet<Tuple2<String, Double>>
-            weightedRatings =
-            ratings.join(weights)
-
-                   // key of the first input
-                   .where(new KeySelector<Rating, String>() {
-                            public String getKey(Rating r) { return r.category; }
-                          })
-
-                   // key of the second input
-                   .equalTo(new KeySelector<Tuple2<String, Double>, String>() {
-                              public String getKey(Tuple2<String, Double> t) { return t.f0; }
-                            });
-{% endhighlight %}
-
-[Back to top](#top)
-
-
-<section id="functions">
-Functions
----------
-
-You can define a user-defined function and pass it to the DataSet
-transformations in several ways:
-
-#### Implementing an interface
-
-The most basic way is to implement one of the provided interfaces:
-
-{% highlight java %}
-class MyMapFunction implements MapFunction<String, Integer> {
-  public Integer map(String value) { return Integer.parseInt(value); }
-});
-data.map (new MyMapFunction());
-{% endhighlight %}
-
-#### Anonymous classes
-
-You can pass a function as an anonmymous class:
-{% highlight java %}
-data.map(new MapFunction<String, Integer> () {
-  public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-
-#### Java 8 Lambdas
-
-***Warning: Lambdas are currently only supported for filter and reduce
-   transformations***
-
-{% highlight java %}
-DataSet<String> data = // [...]
-data.filter(s -> s.startsWith("http://"));
-{% endhighlight %}
-
-{% highlight java %}
-DataSet<Integer> data = // [...]
-data.reduce((i1,i2) -> i1 + i2);
-{% endhighlight %}
-
-#### Rich functions
-
-All transformations that take as argument a user-defined function can
-instead take as argument a *rich* function. For example, instead of
-{% highlight java %}
-class MyMapFunction implements MapFunction<String, Integer> {
-  public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-you can write
-{% highlight java %}
-class MyMapFunction extends RichMapFunction<String, Integer> {
-  public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-and pass the function as usual to a `map` transformation:
-{% highlight java %}
-data.map(new MyMapFunction());
-{% endhighlight %}
-
-Rich functions can also be defined as an anonymous class:
-{% highlight java %}
-data.map (new RichMapFunction<String, Integer>() {
-  public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-
-Rich functions provide, in addition to the user-defined function (map,
-reduce, etc), four methods: `open`, `close`, `getRuntimeContext`, and
-`setRuntimeContext`. These are useful for creating and finalizing
-local state, accessing broadcast variables (see
-[Broadcast Variables](#broadcast_variables), and for accessing runtime
-information such as accumulators and counters (see
-[Accumulators and Counters](#accumulators_counters), and information
-on iterations (see [Iterations](#iterations)).
-
-In particular for the `reduceGroup` transformation, using a rich
-function is the only way to define an optional `combine` function. See
-the
-[transformations documentation](java_api_transformations.html)
-for a complete example.
-
-[Back to top](#top)
-
-
-<section id="types">
-Data Types
-----------
-
-The Java API is strongly typed: All data sets and transformations accept typed elements. This catches type errors very early and supports safe refactoring of programs. The API supports various different data types for the input and output of operators. Both `DataSet` and functions like `MapFunction`, `ReduceFunction`, etc. are parameterized with data types using Java generics in order to ensure type-safety.
-
-There are four different categories of data types, which are treated slightly different:
-
-1. **Regular Types**
-2. **Tuples**
-3. **Values**
-4. **Hadoop Writables**
-
-
-#### Regular Types
-
-Out of the box, the Java API supports all common basic Java types: `Byte`, `Short`, `Integer`, `Long`, `Float`, `Double`, `Boolean`, `Character`, `String`.
-
-Furthermore, you can use the vast majority of custom Java classes. Restrictions apply to classes containing fields that cannot be serialized, like File pointers, I/O streams, or other native resources. Classes that follow the Java Beans conventions work well in general. The following defines a simple example class to illustrate how you can use custom classes:
-
-```java
-public class WordWithCount {
-
-    public String word;
-    public int count;
-
-    public WordCount() {}
-
-    public WordCount(String word, int count) {
-        this.word = word;
-        this.count = count;
-    }
-}
-```
-
-You can use all of those types to parameterize `DataSet` and function implementations, e.g. `DataSet<String>` for a `String` data set or `MapFunction<String, Integer>` for a mapper from `String` to `Integer`.
-
-```java
-// using a basic data type
-DataSet<String> numbers = env.fromElements("1", "2");
-
-numbers.map(new MapFunction<String, Integer>() {
-    @Override
-    public String map(String value) throws Exception {
-        return Integer.parseInt(value);
-    }
-});
-
-// using a custom class
-DataSet<WordCount> wordCounts = env.fromElements(
-    new WordCount("hello", 1),
-    new WordCount("world", 2));
-
-wordCounts.map(new MapFunction<WordCount, Integer>() {
-    @Override
-    public String map(WordCount value) throws Exception {
-        return value.count;
-    }
-});
-```
-
-When working with operators that require a Key for grouping or matching records
-you need to implement a `KeySelector` for your custom type (see
-[Section Defining Keys](#keys)).
-
-```java
-wordCounts.groupBy(new KeySelector<WordCount, String>() {
-    public String getKey(WordCount v) {
-        return v.word;
-    }
-}).reduce(new MyReduceFunction());
-```
-
-#### Tuples
-
-You can use the `Tuple` classes for composite types. Tuples contain a fix number of fields of various types. The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple can be an arbitrary Flink type - including further tuples, resulting in nested tuples. Fields of a Tuple can be accessed directly using the fields `tuple.f4`, or using the generic getter method `tuple.getField(int position)`. The field numbering starts with 0. Note that this stands in contrast to the Scala tuples, but it is more consistent with Java's general indexing.
-
-```java
-DataSet<Tuple2<String, Integer>> wordCounts = env.fromElements(
-    new Tuple2<String, Integer>("hello", 1),
-    new Tuple2<String, Integer>("world", 2));
-
-wordCounts.map(new MapFunction<Tuple2<String, Integer>, Integer>() {
-    @Override
-    public String map(Tuple2<String, Integer> value) throws Exception {
-        return value.f1;
-    }
-});
-```
-
-When working with operators that require a Key for grouping or matching records,
-Tuples let you simply specify the positions of the fields to be used as key. You can specify more
-than one position to use composite keys (see [Section Data Transformations](#transformations)).
-
-```java
-wordCounts
-    .groupBy(0)
-    .reduce(new MyReduceFunction());
-```
-
-In order to access fields more intuitively and to generate more readable code, it is also possible to extend a subclass of `Tuple`. You can add getters and setters with custom names that delegate to the field positions. See this {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java "example" %} for an illustration how to make use of that mechanism.
-
-
-#### Values
-
-*Value* types describe their serialization and deserialization manually. Instead of going through a general purpose serialization framework, they provide custom code for those operations by means implementing the `org.apache.flinktypes.Value` interface with the methods `read` and `write`. Using a *Value* type is reasonable when general purpose serialization would be highly inefficient. An example would be a data type that implements a sparse vector of elements as an array. Knowing that the array is mostly zero, one can use a special encoding for the non-zero elements, while the general purpose serialization would simply write all array elements.
-
-The `org.apache.flinktypes.CopyableValue` interface supports manual internal cloning logic in a similar way.
-
-Flink comes with pre-defined Value types that correspond to Java's basic data types. (`ByteValue`, `ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`, `BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can be altered, allowing programmers to reuse objects and take pressure off the garbage collector. 
-
-
-#### Hadoop Writables
-
-You can use types that implement the `org.apache.hadoop.Writable` interface. The serialization logic defined in the `write()`and `readFields()` methods will be used for serialization.
-
-
-#### Type Erasure & Type Inferrence
-
-The Java compiler throws away much of the generic type information after the compilation. This is known as *type erasure* in Java. It means that at runtime, an instance of an object does not know its generic type any more. For example, instances of `DataSet<String>` and `DataSet<Long>` look the same to the JVM.
-
-Flink requires type information at the time when it prepares the program for execution (when the main method of the program is called). The Flink Java API tries to reconstruct the type information that was thrown away in various ways and store it explicitly in the data sets and operators. You can retrieve the type via `DataSet.getType()`. The method returns an instance of `TypeInformation`, which is Flink's internal way of representing types.
-
-The type inference has its limits and needs the "cooperation" of the programmer in some cases. Examples for that are methods that create data sets from collections, such as `ExecutionEnvironment.fromCollection(),` where you can pass an argument that describes the type. But also generic functions like `MapFunction<I, O>` may need extra type information.
-
-The {% gh_link /flink-java/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java "ResultTypeQueryable" %} interface can be implemented by input formats and functions to tell the API explicitly about their return type. The *input types* that the functions are invoked with can usually be inferred by the result types of the previous operations.
-
-[Back to top](#top)
-
-
-<section id="data_sources">
-Data Sources
-------------
-
-Data sources create the initial data sets, such as from files or from Java collections. The general mechanism of of creating data sets is abstracted behind an {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java "InputFormat" %}. Flink comes with several built-in formats to create data sets from common file formats. Many of them have shortcut methods on the *ExecutionEnvironment*.
-
-File-based:
-
-- `readTextFile(path)` / `TextInputFormat` - Reads files line wise and returns them as Strings.
-- `readTextFileWithValue(path)` / `TextValueInputFormat` - Reads files line wise and returns them as StringValues. StringValues are mutable strings.
-- `readCsvFile(path)` / `CsvInputFormat` - Parses files of comma (or another char) delimited fields. Returns a DataSet of tuples. Supports the basic java types and their Value counterparts as field types.
-
-Collection-based:
-
-- `fromCollection(Collection)` - Creates a data set from the Java Java.util.Collection. All elements in the collection must be of the same type.
-- `fromCollection(Iterator, Class)` - Creates a data set from an iterator. The class specifies the data type of the elements returned by the iterator.
-- `fromElements(T ...)` - Creates a data set from the given sequence of objects. All objects must be of the same type.
-- `fromParallelCollection(SplittableIterator, Class)` - Creates a data set from an iterator, in parallel. The class specifies the data type of the elements returned by the iterator.
-- `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in parallel.
-
-Generic:
-
-- `createInput(path)` / `InputFormat` - Accepts a generic input format.
-
-**Examples**
-
-```java
-ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-// read text file from local files system
-DataSet<String> localLines = env.readTextFile("file:///path/to/my/textfile");
-
-// read text file from a HDFS running at nnHost:nnPort
-DataSet<String> hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile");
-
-// read a CSV file with three fields
-DataSet<Tuple3<Integer, String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
-	                       .types(Integer.class, String.class, Double.class);
-
-// read a CSV file with five fields, taking only two of them
-DataSet<Tuple2<String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
-                               .includeFields("10010")  // take the first and the fourth fild
-	                       .types(String.class, Double.class);
-
-// create a set from some given elements
-DataSet<String> value = env.fromElements("Foo", "bar", "foobar", "fubar");
-
-// generate a number sequence
-DataSet<Long> numbers = env.generateSequence(1, 10000000);
-
-// Read data from a relational database using the JDBC input format
-DataSet<Tuple2<String, Integer> dbData = 
-    env.createInput(
-      // create and configure input format
-      JDBCInputFormat.buildJDBCInputFormat()
-                     .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
-                     .setDBUrl("jdbc:derby:memory:persons")
-                     .setQuery("select name, age from persons")
-                     .finish(),
-      // specify type information for DataSet
-      new TupleTypeInfo(Tuple2.class, STRING_TYPE_INFO, INT_TYPE_INFO)
-    );
-
-// Note: Flink's program compiler needs to infer the data types of the data items which are returned by an InputFormat. If this information cannot be automatically inferred, it is necessary to manually provide the type information as shown in the examples above.
-```
-
-[Back to top](#top)
-
-
-<section id="data_sinks">
-Data Sinks
-----------
-
-Data sinks consume DataSets and are used to store or return them. Data sink operations are described using an {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java "OutputFormat" %}. Flink comes with a variety of built-in output formats that
-are encapsulated behind operations on the DataSet type:
-
-- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are obtained by calling the *toString()* method of each element.
-- `writeAsFormattedText()` / `TextOutputFormat` - Write elements line-wise as Strings. The Strings are obtained by calling a user-defined *format()* method for each element.
-- `writeAsCsv` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects.
-- `print()` / `printToErr()` - Prints the *toString()* value of each element on the standard out / strandard error stream.
-- `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports custom object-to-bytes conversion.
-- `output()`/ `OutputFormat` - Most generic output method, for data sinks that are not file based (such as storing the result in a database).
-
-A DataSet can be input to multiple operations. Programs can write or print a data set and at the same time run additional transformations on them.
-
-**Examples**
-
-Standard data sink methods:
-
-```java
-// text data 
-DataSet<String> textData = // [...]
-
-// write DataSet to a file on the local file system
-textData.writeAsText("file:///my/result/on/localFS");
-
-// write DataSet to a file on a HDFS with a namenode running at nnHost:nnPort
-textData.writeAsText("hdfs://nnHost:nnPort/my/result/on/localFS");
-
-// write DataSet to a file and overwrite the file if it exists
-textData.writeAsText("file:///my/result/on/localFS", WriteMode.OVERWRITE);
-
-// tuples as lines with pipe as the separator "a|b|c"
-DataSet<Tuple3<String, Integer, Double>> values = // [...]
-values.writeAsCsv("file:///path/to/the/result/file", "\n", "|");
-
-// this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines
-values.writeAsText("file:///path/to/the/result/file");
-
-// this wites values as strings using a user-defined TextFormatter object
-values.writeAsFormattedText("file:///path/to/the/result/file", new TextFormatter<Tuple2<Integer, Integer>>() {
-    public String format (Tuple2<Integer, Integer> value) {
-        return value.f1 + " - " + value.f0;
-    }});
-```
-
-Using a custom output format:
-
-```java
-DataSet<Tuple3<String, Integer, Double>> myResult = [...]
-
-// write Tuple DataSet to a relational database
-myResult.output(
-    // build and configure OutputFormat
-    JDBCOutputFormat.buildJDBCOutputFormat()
-                    .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
-                    .setDBUrl("jdbc:derby:memory:persons")
-                    .setQuery("insert into persons (name, age, height) values (?,?,?)")
-                    .finish()
-    );
-```
-
-[Back to top](#top)
-
-
-<section id="debugging">
-Debugging
----------
-
-Before running a data analysis program on a large data set in a distributed cluster, it is a good idea to make sure that the implemented algorithm works as desired. Hence, implementing data analysis programs is usually an incremental process of checking results, debugging, and improving. 
-
-<p>
-Flink provides a few nice features to significantly ease the development process of data analysis programs by supporting local debugging from within an IDE, injection of test data, and collection of result data. This section give some hints how to ease the development of Flink programs.
-</p>
-
-### Local Execution Environment
-
-A `LocalEnvironment` starts a Flink system within the same JVM process it was created in. If you start the LocalEnvironement from an IDE, you can set breakpoint in your code and easily debug your program. 
-
-<p>
-A LocalEnvironment is created and used as follows:
-</p>
-
-```java
-final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-
-DataSet<String> lines = env.readTextFile(pathToTextFile);
-// build your program
-
-env.execute();
-
-```
-
-### Collection Data Sources and Sinks
-
-Providing input for an analysis program and checking its output is cumbersome done by creating input files and reading output files. Flink features special data sources and sinks which are backed by Java collections to ease testing. Once a program has been tested, the sources and sinks can be easily replaced by sources and sinks that read from / write to external data stores such as HDFS.
-
-Collection data sources can be used as follows:
-
-```java
-final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-
-// Create a DataSet from a list of elements
-DataSet<Integer> myInts = env.fromElements(1, 2, 3, 4, 5);
-
-// Create a DataSet from any Java collection
-List<Tuple2<String, Integer>> data = ...
-DataSet<Tuple2<String, Integer>> myTuples = env.fromCollection(data);
-
-// Create a DataSet from an Iterator
-Iterator<Long> longIt = ...
-DataSet<Long> myLongs = env.fromCollection(longIt, Long.class);
-```
-
-**Note:** Currently, the collection data source requires that data types and iterators implement `Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of parallelism = 1).
-
-A collection data sink is specified as follows:
-
-```java
-DataSet<Tuple2<String, Integer>> myResult = ...
-
-List<Tuple2<String, Integer>> outData = new ArrayList<Tuple2<String, Integer>>();
-myResult.output(new LocalCollectionOutputFormat(outData));
-```
-
-**Note:** Currently, the collection data sink is restricted to local execution, as a debugging tool.
-
-[Back to top](#top)
-
-
-<section id="iterations">
-Iteration Operators
--------------------
-
-Iterations implement loops in Flink programs. The iteration operators encapsulate a part of the program and execute it repeatedly, feeding back the result of one iteration (the partial solution) into the next iteration. There are two types of iterations in Flink: **BulkIteration** and **DeltaIteration**.
-
-This section provides quick examples on how to use both operators. Check out the [Introduction to Iterations](iterations.html) page for a more detailed introduction.
-
-#### Bulk Iterations
-
-To create a BulkIteration call the `iterate(int)` method of the `DataSet` the iteration should start at. This will return an `IterativeDataSet`, which can be transformed with the regular operators. The single argument to the iterate call specifies the maximum number of iterations.
-
-To specify the end of an iteration call the `closeWith(DataSet)` method on the `IterativeDataSet` to specify which transformation should be fed back to the next iteration. You can optionally specify a termination criterion with `closeWith(DataSet, DataSet)`, which evaluates the second DataSet and terminates the iteration, if this DataSet is empty. If no termination criterion is specified, the iteration terminates after the given maximum number iterations.
-
-The following example iteratively estimates the number Pi. The goal is to count the number of random points, which fall into the unit circle. In each iteration, a random point is picked. If this point lies inside the unit circle, we increment the count. Pi is then estimated as the resulting count divided by the number of iterations multiplied by 4.
-
-{% highlight java %}
-final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-// Create initial IterativeDataSet
-IterativeDataSet<Integer> initial = env.fromElements(0).iterate(10000);
-
-DataSet<Integer> iteration = initial.map(new MapFunction<Integer, Integer>() {
-    @Override
-    public Integer map(Integer i) throws Exception {
-        double x = Math.random();
-        double y = Math.random();
-
-        return i + ((x * x + y * y < 1) ? 1 : 0);
-    }
-});
-
-// Iteratively transform the IterativeDataSet
-DataSet<Integer> count = initial.closeWith(iteration);
-
-count.map(new MapFunction<Integer, Double>() {
-    @Override
-    public Double map(Integer count) throws Exception {
-        return count / (double) 10000 * 4;
-    }
-}).print();
-
-env.execute("Iterative Pi Example");
-{% endhighlight %}
-
-You can also check out the {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java "K-Means example" %}, which uses a BulkIteration to cluster a set of unlabeled points.
-
-#### Delta Iterations
-
-Delta iterations exploit the fact that certain algorithms do not change every data point of the solution in each iteration.
-
-In addition to the partial solution that is fed back (called workset) in every iteration, delta iterations maintain state across iterations (called solution set), which can be updated through deltas. The result of the iterative computation is the state after the last iteration. Please refer to the [Introduction to Iterations](iterations.html) for an overview of the basic principle of delta iterations.
-
-Defining a DeltaIteration is similar to defining a BulkIteration. For delta iterations, two data sets form the input to each iteration (workset and solution set), and two data sets are produced as the result (new workset, solution set delta) in each iteration.
-
-To create a DeltaIteration call the `iterateDelta(DataSet, int, int)` (or `iterateDelta(DataSet, int, int[])` respectively). This method is called on the initial solution set. The arguments are the initial delta set, the maximum number of iterations and the key positions. The returned `DeltaIteration` object gives you access to the DataSets representing the workset and solution set via the methods `iteration.getWorket()` and `iteration.getSolutionSet()`.
-
-Below is an example for the syntax of a delta iteration
-
-```java
-// read the initial data sets
-DataSet<Tuple2<Long, Double>> initialSolutionSet = // [...]
-
-DataSet<Tuple2<Long, Double>> initialDeltaSet = // [...]
-
-int maxIterations = 100;
-int keyPosition = 0;
-
-DeltaIteration<Tuple2<Long, Double>, Tuple2<Long, Double>> iteration = initialSolutionSet
-    .iterateDelta(initialDeltaSet, maxIterations, keyPosition);
-
-DataSet<Tuple2<Long, Double>> candidateUpdates = iteration.getWorkset()
-    .groupBy(1)
-    .reduceGroup(new ComputeCandidateChanges());
-
-DataSet<Tuple2<Long, Double>> deltas = candidateUpdates
-    .join(iteration.getSolutionSet())
-    .where(0)
-    .equalTo(0)
-    .with(new CompareChangesToCurrent());
-
-DataSet<Tuple2<Long, Double>> nextWorkset = deltas
-    .filter(new FilterByThreshold());
-
-iteration.closeWith(deltas, nextWorkset)
-	.writeAsCsv(outputPath);
-```
-
-[Back to top](#top)
-
-
-<section id="annotations">
-Semantic Annotations
------------
-
-Semantic Annotations give hints about the behavior of a function by telling the system which fields in the input are accessed and which are constant between input and output data of a function (copied but not modified). Semantic annotations are a powerful means to speed up execution, because they allow the system to reason about reusing sort orders or partitions across multiple operations. Using semantic annotations may eventually save the program from unnecessary data shuffling or unnecessary sorts.
-
-Semantic annotations can be attached to functions through Java annotations, or passed as arguments when invoking a function on a DataSet. The following example illustrates that:
-
-```java
-@ConstantFields("1")
-public class DivideFirstbyTwo implements MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
-  @Override
-  public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) {
-    value.f0 /= 2;
-    return value;
-  }
-}
-```
-
-The following annotations are currently available:
-
-* `@ConstantFields`: Declares constant fields (forwarded/copied) for functions with a single input data set (Map, Reduce, Filter, ...).
-
-* `@ConstantFieldsFirst`: Declares constant fields (forwarded/copied) for functions with a two input data sets (Join, CoGroup, ...), with respect to the first input data set.
-
-* `@ConstantFieldsSecond`: Declares constant fields (forwarded/copied) for functions with a two input data sets (Join, CoGroup, ...), with respect to the first second data set.
-
-* `@ConstantFieldsExcept`: Declares that all fields are constant, except for the specified fields. Applicable to functions with a single input data set.
-
-* `@ConstantFieldsFirstExcept`: Declares that all fields of the first input are constant, except for the specified fields. Applicable to functions with a two input data sets.
-
-* `@ConstantFieldsSecondExcept`: Declares that all fields of the second input are constant, except for the specified fields. Applicable to functions with a two input data sets.
-
-*(Note: The system currently evaluated annotations only on Tuple data types. This will be extended in the next versions)*
-
-**Note**: It is important to be conservative when providing annotations. Only annotate fields, when they are always constant for every call to the function. Otherwise the system has incorrect assumptions about the execution and the execution may produce wrong results. If the behavior of the operator is not clearly predictable, no annotation should be provided.
-
-[Back to top](#top)
-
-
-<section id="broadcast_variables">
-Broadcast Variables
--------------------
-
-Broadcast variables allow you to make a data set available to all parallel instances of an operation, in addition to the regular input of the operation. This is useful
-for auxiliary data sets, or data-dependent parameterization. The data set will then be accessible at the operator as an `Collection<T>`.
-
-- **Broadcast**: broadcast sets are registered by name via `withBroadcastSet(DataSet, String)`, and
-- **Access**: accessible via `getRuntimeContext().getBroadcastVariable(String)` at the target operator.
-
-```java
-// 1. The DataSet to be broadcasted
-DataSet<Integer> toBroadcast = env.fromElements(1, 2, 3);
-
-DataSet<String> data = env.fromElements("a", "b");
-
-data.map(new MapFunction<String, String>() {
-    @Override
-    public void open(Configuration parameters) throws Exception {
-      // 3. Access the broadcasted DataSet as a Collection
-      Collection<Integer> broadcastSet = getRuntimeContext().getBroadcastVariable("broadcastSetName");
-    }
-
-
-    @Override
-    public String map(String value) throws Exception {
-        ...
-    }
-}).withBroadcastSet(toBroadcast, "broadcastSetName"); // 2. Broadcast the DataSet
-```
-
-Make sure that the names (`broadcastSetName` in the previous example) match when registering and accessing broadcasted data sets. For a complete example program, have a look at
-{% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java#L96 "KMeans Algorithm" %}.
-
-**Note**: As the content of broadcast variables is kept in-memory on each node, it should not become too large. For simpler things like scalar values you can simply make parameters part of the closure of a function, or use the `withParameters(...)` method to pass in a configuration.
-
-[Back to top](#top)
-
-
-<section id="packaging">
-Program Packaging & Distributed Execution
------------------------------------------
-
-As described in the [program skeleton](#skeleton) section, Flink programs can be executed on clusters by using the `RemoteEnvironment`. Alternatively, programs can be packaged into JAR Files (Java Archives) for execution. Packaging the program is a prerequisite to executing them through the [command line interface](cli.html) or the [web interface](web_client.html).
-
-#### Packaging Programs
-
-To support execution from a packaged JAR file via the command line or web interface, a program must use the environment obtained by `ExecutionEnvironment.getExecutionEnvironment()`. This environment will act as the cluster's environment when the JAR is submitted to the command line or web interface. If the Flink program is invoked differently than through these interfaces, the environment will act like a local environment.
-
-To package the program, simply export all involved classes as a JAR file. The JAR file's manifest must point to the class that contains the program's *entry point* (the class with the `public void main(String[])` method). The simplest way to do this is by putting the *main-class* entry into the manifest (such as `main-class: org.apache.flinkexample.MyProgram`). The *main-class* attribute is the same one that is used by the Java Virtual Machine to find the main method when executing a JAR files through the command `java -jar pathToTheJarFile`. Most IDEs offer to include that attribute automatically when exporting JAR files.
-
-
-#### Packaging Programs through Plans
-
-Additionally, the Java API supports packaging programs as *Plans*. This method resembles the way that the *Scala API* packages programs. Instead of defining a progam in the main method and calling `execute()` on the environment, plan packaging returns the *Program Plan*, which is a description of the program's data flow. To do that, the program must implement the `org.apache.flinkapi.common.Program` interface, defining the `getPlan(String...)` method. The strings passed to that method are the command line arguments. The program's plan can be created from the environment via the `ExecutionEnvironment#createProgramPlan()` method. When packaging the program's plan, the JAR manifest must point to the class implementing the `org.apache.flinkapi.common.Program` interface, instead of the class with the main method.
-
-
-#### Summary
-
-The overall procedure to invoke a packaged program is as follows:
-
-  1. The JAR's manifest is searched for a *main-class* or *program-class* attribute. If both attributes are found, the *program-class* attribute takes precedence over the *main-class* attribute. Both the command line and the web interface support a parameter to pass the entry point class name manually for cases where the JAR manifest contains neither attribute.
-  2. If the entry point class implements the `org.apache.flinkapi.common.Program`, then the system calls the `getPlan(String...)` method to obtain the program plan to execute. The `getPlan(String...)` method was the only possible way of defining a program in the *Record API* (see [0.4 docs](http://stratosphere.eu/docs/0.4/)) and is also supported in the new Java API.
-  3. If the entry point class does not implement the `org.apache.flinkapi.common.Program` interface, the system will invoke the main method of the class.
-
-[Back to top](#top)
-
-
-<section id="accumulators_counters">
-Accumulators & Counters
----------------------------
-
-Accumulators are simple constructs with an **add operation** and a **final accumulated result**, which is available after the job ended.
-
-The most straightforward accumulator is a **counter**: You can increment it using the ```Accumulator.add(V value)``` method. At the end of the job Flink will sum up (merge) all partial results and send the result to the client. Since accumulators are very easy to use, they can be useful during debugging or if you quickly want to find out more about your data.
-
-Flink currently has the following **built-in accumulators**. Each of them implements the {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %} interface.
-
-- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java "__IntCounter__" %}, {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java "__LongCounter__" %} and {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java "__DoubleCounter__" %}: See below for an example using a counter.
-- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java "__Histogram__" %}: A histogram implementation for a discrete number of bins. Internally it is just a map from Integer to Integer. You can use this to compute distributions of values, e.g. the distribution of words-per-line for a word count program.
-
-__How to use accumulators:__
-
-First you have to create an accumulator object (here a counter) in the operator function where you want to use it. Operator function here refers to the (anonymous inner)
-class implementing the user defined code for an operator.
-
-    private IntCounter numLines = new IntCounter();
-
-Second you have to register the accumulator object, typically in the ```open()``` method of the operator function. Here you also define the name.
-
-    getRuntimeContext().addAccumulator("num-lines", this.numLines);
-
-You can now use the accumulator anywhere in the operator function, including in the ```open()``` and ```close()``` methods.
-
-    this.numLines.add(1);
-
-The overall result will be stored in the ```JobExecutionResult``` object which is returned when running a job using the Java API (currently this only works if the execution waits for the completion of the job).
-
-    myJobExecutionResult.getAccumulatorResult("num-lines")
-
-All accumulators share a single namespace per job. Thus you can use the same accumulator in different operator functions of your job. Flink will internally merge all accumulators with the same name.
-
-A note on accumulators and iterations: Currently the result of accumulators is only available after the overall job ended. We plan to also make the result of the previous iteration available in the next iteration. You can use {% gh_link /flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java#L98 "Aggregators" %} to compute per-iteration statistics and base the termination of iterations on such statistics.
-
-__Custom accumulators:__
-
-To implement your own accumulator you simply have to write your implementation of the Accumulator interface. Feel free to create a pull request if you think your custom accumulator should be shipped with Flink.
-
-You have the choice to implement either {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %} or {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java "SimpleAccumulator" %}. ```Accumulator<V,R>``` is most flexible: It defines a type ```V``` for the value to add, and a result type ```R``` for the final result. E.g. for a histogram, ```V``` is a number and ```R``` is a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.
-
-[Back to top](#top)
-
-<section id="parallelism">
-Parallel Execution
----------
-
-This section describes how the parallel execution of programs can be configured in Flink. A Flink program consists of multiple tasks (operators, data sources, and sinks). A task is split into several parallel instances for execution and each parallel instance processes a subset of the task's input data. The number of parallel instances of a task is called its *parallelism* or *degree of parallelism (DOP)*.
-
-The degree of parallelism of a task can be specified in Flink on different levels.
-
-### Operator Level
-The parallelism of an individual operator, data source, or data sink can be defined by calling its `setParallelism()` method. 
-For example, the degree of parallelism of the `Sum` operator in the [WordCount](#example) example program can be set to `5` as follows :
-
-
-```java
-final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-DataSet<String> text = [...]
-DataSet<Tuple2<String, Integer>> wordCounts = text
-    .flatMap(new LineSplitter())
-    .groupBy(0)
-    .sum(1).setParallelism(5);
-wordCounts.print();
-
-env.execute("Word Count Example");
-```
-
-### Execution Environment Level
-
-Flink programs are executed in the context of an [execution environmentt](#program-skeleton). An execution environment defines a default parallelism for all operators, data sources, and data sinks it executes. Execution environment parallelism can be overwritten by explicitly configuring the parallelism of an operator.
-
-The default parallelism of an execution environment can be specified by calling the `setDefaultLocalParallelism()` method. To execute all operators, data sources, and data sinks of the [WordCount](#example) example program with a parallelism of `3`, set the default parallelism of the execution environment as follows:
-
-```java
-final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-env.setDegreeOfParallelism(3);
-
-DataSet<String> text = [...]
-DataSet<Tuple2<String, Integer>> wordCounts = [...]
-wordCounts.print();
-
-env.execute("Word Count Example");
-```
-
-### System Level
-
-A system-wide default parallelism for all execution environments can be defined by setting the `parallelization.degree.default` property in `./conf/flink-conf.yaml`. See the [Configuration]({{site.baseurl}}/config.html) documentation for details.
-
-[Back to top](#top)
-
-<section id="execution_plan">
-Execution Plans
----------------
-
-Depending on various parameters such as data size or number of machines in the cluster, Flink's optimizer automatically chooses an execution strategy for your program. In many cases, it can be useful to know how exactly Flink will execute your program.
-
-__Plan Visualization Tool__
-
-Flink 0.5 comes packaged with a visualization tool for execution plans. The HTML document containing the visualizer is located under ```tools/planVisualizer.html```. It takes a JSON representation of the job execution plan and visualizes it as a graph with complete annotations of execution strategies.
-
-The following code shows how to print the execution plan JSON from your program:
-
-    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-    ...
-
-    System.out.println(env.getExecutionPlan());
-
-
-To visualize the execution plan, do the following:
-
-1. **Open** ```planVisualizer.html``` with your web browser,
-2. **Paste** the JSON string into the text field, and
-3. **Press** the draw button.
-
-After these steps, a detailed execution plan will be visualized.
-
-<img alt="A flink job execution graph." src="img/plan_visualizer2.png" width="80%">
-
-
-__Web Interface__
-
-Flink offers a web interface for submitting and executing jobs. If you choose to use this interface to submit your packaged program, you have the option to also see the plan visualization.
-
-The script to start the webinterface is located under ```bin/start-webclient.sh```. After starting the webclient (per default on **port 8080**), your program can be uploaded and will be added to the list of available programs on the left side of the interface.
-
-You are able to specify program arguments in the textbox at the bottom of the page. Checking the plan visualization checkbox shows the execution plan before executing the actual program.
-
-[Back to top](#top)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/java_api_quickstart.md b/docs/java_api_quickstart.md
index d51088a..b202304 100644
--- a/docs/java_api_quickstart.md
+++ b/docs/java_api_quickstart.md
@@ -2,13 +2,16 @@
 title: "Quickstart: Java API"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Start working on your Flink Java program in a few simple steps.
 
 
-# Requirements
+## Requirements
 The only requirements are working __Maven 3.0.4__ (or higher) and __Java 6.x__ (or higher) installations.
 
-# Create Project
+## Create Project
 Use one of the following commands to __create a project__:
 
 <ul class="nav nav-tabs" style="border-bottom: none;">
@@ -32,7 +35,7 @@ Use one of the following commands to __create a project__:
     </div>
 </div>
 
-# Inspect Project
+## Inspect Project
 There will be a new directory in your working directory. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId.
 
 The sample project is a __Maven project__, which contains two classes. _Job_ is a basic skeleton program and _WordCountJob_ a working example. Please note that the _main_ method of both classes allow you to start Flink in a development/testing mode.
@@ -42,28 +45,31 @@ We recommend to __import this project into your IDE__ to develop and test it. If
 
 A note to Mac OS X users: The default JVM heapsize for Java is too small for Flink. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse.
 
-# Build Project
+## Build Project
 If you want to __build your project__, go to your project directory and issue the `mvn clean package` command. You will __find a jar__ that runs on every Flink cluster in `target/flink-project-0.1-SNAPSHOT.jar`.
 
-# Next Steps
+## Next Steps
 Write your application!
 
 The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occurs in all Wikipedia texts.
 
 __Sample Input__:
-```bash
+
+~~~bash
 big data is big
-```
+~~~
 
 __Sample Output__:
-```bash
+
+~~~bash
 big 2
 data 1
 is 1
-```
+~~~
+
 The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and writes the prints the resulting words and counts to std-out.
 
-```java
+~~~java
 public class WordCount {
   
   public static void main(String[] args) throws Exception {
@@ -93,11 +99,11 @@ public class WordCount {
     env.execute("WordCount Example");
   }
 }
-```
+~~~
 
 The operations are defined by specialized classes, here the LineSplitter class.
 
-```java
+~~~java
 public class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer>> {
 
   @Override
@@ -113,7 +119,8 @@ public class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer
     }
   }
 }
-```
+~~~
+
 {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java "Check GitHub" %} for the full example code.
 
 For a complete overview over our Java API, have a look at the [API Documentation](java_api_guide.html) and [further example programs](java_api_examples.html). If you have any trouble, ask on our [Mailing List](http://mail-archives.apache.org/mod_mbox/incubator-flink-dev/). We are happy to provide help.


[20/60] Refactor TupleTypeInfo and add GenericPairComparator

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
index 1c777fc..58a9a2a 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
@@ -19,7 +19,6 @@
 package org.apache.flink.api.java.typeutils.runtime;
 
 import java.io.IOException;
-import java.util.Arrays;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple;
@@ -28,49 +27,16 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.NullFieldException;
 
 
-public final class TupleSerializer<T extends Tuple> extends TypeSerializer<T> {
+public final class TupleSerializer<T extends Tuple> extends TupleSerializerBase<T> {
 
 	private static final long serialVersionUID = 1L;
 	
-	
-	private final Class<T> tupleClass;
-	
-	private final TypeSerializer<Object>[] fieldSerializers;
-	
-	private final int arity;
-	
-	private final boolean stateful;
-	
-	
 	@SuppressWarnings("unchecked")
 	public TupleSerializer(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers) {
-		this.tupleClass = tupleClass;
-		this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
-		this.arity = fieldSerializers.length;
-		
-		boolean stateful = false;
-		for (TypeSerializer<?> ser : fieldSerializers) {
-			if (ser.isStateful()) {
-				stateful = true;
-				break;
-			}
-		}
-		this.stateful = stateful;
-	}
-	
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
+		super(tupleClass, fieldSerializers);
 	}
 
 	@Override
-	public boolean isStateful() {
-		return this.stateful;
-	}
-	
-	
-	@Override
 	public T createInstance() {
 		try {
 			T t = tupleClass.newInstance();
@@ -97,12 +63,6 @@ public final class TupleSerializer<T extends Tuple> extends TypeSerializer<T> {
 	}
 
 	@Override
-	public int getLength() {
-		return -1;
-	}
-
-
-	@Override
 	public void serialize(T value, DataOutputView target) throws IOException {
 		for (int i = 0; i < arity; i++) {
 			Object o = value.getField(i);
@@ -122,33 +82,4 @@ public final class TupleSerializer<T extends Tuple> extends TypeSerializer<T> {
 		}
 		return reuse;
 	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		for (int i = 0; i < arity; i++) {
-			fieldSerializers[i].copy(source, target);
-		}
-	}
-	
-	@Override
-	public int hashCode() {
-		int hashCode = arity * 47;
-		for (TypeSerializer<?> ser : this.fieldSerializers) {
-			hashCode = (hashCode << 7) | (hashCode >>> -7);
-			hashCode += ser.hashCode();
-		}
-		return hashCode;
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		if (obj != null && obj instanceof TupleSerializer) {
-			TupleSerializer<?> otherTS = (TupleSerializer<?>) obj;
-			return (otherTS.tupleClass == this.tupleClass) && 
-					Arrays.deepEquals(this.fieldSerializers, otherTS.fieldSerializers);
-		}
-		else {
-			return false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
new file mode 100644
index 0000000..69133b6
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+
+public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
+
+	protected final Class<T> tupleClass;
+
+	protected final TypeSerializer<Object>[] fieldSerializers;
+
+	protected final int arity;
+
+	protected final boolean stateful;
+
+
+	@SuppressWarnings("unchecked")
+	public TupleSerializerBase(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers) {
+		this.tupleClass = tupleClass;
+		this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
+		this.arity = fieldSerializers.length;
+		
+		boolean stateful = false;
+		for (TypeSerializer<?> ser : fieldSerializers) {
+			if (ser.isStateful()) {
+				stateful = true;
+				break;
+			}
+		}
+		this.stateful = stateful;
+	}
+	
+	
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public boolean isStateful() {
+		return this.stateful;
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		for (int i = 0; i < arity; i++) {
+			fieldSerializers[i].copy(source, target);
+		}
+	}
+	
+	@Override
+	public int hashCode() {
+		int hashCode = arity * 47;
+		for (TypeSerializer<?> ser : this.fieldSerializers) {
+			hashCode = (hashCode << 7) | (hashCode >>> -7);
+			hashCode += ser.hashCode();
+		}
+		return hashCode;
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj instanceof TupleSerializerBase) {
+			TupleSerializerBase<?> otherTS = (TupleSerializerBase<?>) obj;
+			return (otherTS.tupleClass == this.tupleClass) && 
+					Arrays.deepEquals(this.fieldSerializers, otherTS.fieldSerializers);
+		}
+		else {
+			return false;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
index 3ca3831..a912844 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java
@@ -46,8 +46,11 @@ public class ValueComparator<T extends Value & Comparable<T>> extends TypeCompar
 	private transient T tempReference;
 	
 	private transient Kryo kryo;
-	
-	
+
+	private final Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {this};
+
 	public ValueComparator(boolean ascending, Class<T> type) {
 		this.type = type;
 		this.ascendingComparison = ascending;
@@ -83,7 +86,7 @@ public class ValueComparator<T extends Value & Comparable<T>> extends TypeCompar
 	}
 	
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		if (reference == null) {
 			reference = InstantiationUtil.instantiate(type, Value.class);
 		}
@@ -140,6 +143,17 @@ public class ValueComparator<T extends Value & Comparable<T>> extends TypeCompar
 			this.kryo.register(type);
 		}
 	}
+
+	@Override
+	public Object[] extractKeys(T record) {
+		extractedKey[0] = record;
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	// unsupported normalization

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
index 6d983b3..49774dc 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java
@@ -43,7 +43,11 @@ public class WritableComparator<T extends Writable & Comparable<T>> extends Type
 	private transient T tempReference;
 	
 	private transient Kryo kryo;
-	
+
+	private final Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {this};
+
 	public WritableComparator(boolean ascending, Class<T> type) {
 		this.type = type;
 		this.ascendingComparison = ascending;
@@ -79,7 +83,7 @@ public class WritableComparator<T extends Writable & Comparable<T>> extends Type
 	}
 	
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		ensureReferenceInstantiated();
 		ensureTempReferenceInstantiated();
 		
@@ -123,6 +127,16 @@ public class WritableComparator<T extends Writable & Comparable<T>> extends Type
 	public TypeComparator<T> duplicate() {
 		return new WritableComparator<T>(ascendingComparison, type);
 	}
+
+	@Override
+	public Object[] extractKeys(T record) {
+		extractedKey[0] = record;
+		return extractedKey;
+	}
+
+	@Override public TypeComparator[] getComparators() {
+		return comparators;
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	// unsupported normalization

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/record/RecordComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/record/RecordComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/record/RecordComparator.java
index 6591502..bf8d56c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/record/RecordComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/record/RecordComparator.java
@@ -264,7 +264,7 @@ public final class RecordComparator extends TypeComparator<Record> {
 	}
 	
 	@Override
-	public int compare(DataInputView source1, DataInputView source2) throws IOException {
+	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
 		this.temp1.read(source1);
 		this.temp2.read(source2);
 		
@@ -391,6 +391,18 @@ public final class RecordComparator extends TypeComparator<Record> {
 	}
 
 	@Override
+	public Object[] extractKeys(Record record) {
+		throw new UnsupportedOperationException("Record does not support extactKeys and " +
+				"getComparators. This cannot be used with the GenericPairComparator.");
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		throw new UnsupportedOperationException("Record does not support extactKeys and " +
+				"getComparators. This cannot be used with the GenericPairComparator.");
+	}
+
+	@Override
 	public boolean supportsCompareAgainstReference() {
 		return true;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
index 0e06226..1c7b6a9 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
@@ -146,7 +146,7 @@ public class TypeInfoParserTest {
 		Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, ((TupleTypeInfo<?>)ti).getTypeAt(1));
 		
 		ti = TypeInfoParser.parse("Tuple3<Tuple1<String>, Tuple1<Integer>, Tuple2<Long, Long>>");
-		Assert.assertEquals("Tuple3<Tuple1<String>, Tuple1<Integer>, Tuple2<Long, Long>>", ti.toString());
+		Assert.assertEquals("Java Tuple3<Java Tuple1<String>, Java Tuple1<Integer>, Java Tuple2<Long, Long>>", ti.toString());
 	}
 	
 	@Test
@@ -190,13 +190,13 @@ public class TypeInfoParserTest {
 		Assert.assertTrue(((ObjectArrayTypeInfo<?, ?>) ti2).getComponentInfo() instanceof TupleTypeInfo);
 		
 		TypeInformation<?> ti3 = TypeInfoParser.parse("Tuple2<Integer[],Double>[]");
-		Assert.assertEquals("ObjectArrayTypeInfo<Tuple2<BasicArrayTypeInfo<Integer>, Double>>", ti3.toString());
+		Assert.assertEquals("ObjectArrayTypeInfo<Java Tuple2<BasicArrayTypeInfo<Integer>, Double>>", ti3.toString());
 	}
 	
 	@Test
 	public void testLargeMixedTuple() {
 		TypeInformation<?> ti = TypeInfoParser.parse("org.apache.flink.api.java.tuple.Tuple4<Double,java.lang.Class[],StringValue,Tuple1<int>>[]");
-		Assert.assertEquals("ObjectArrayTypeInfo<Tuple4<Double, ObjectArrayTypeInfo<GenericType<java.lang.Class>>, ValueType<org.apache.flink.types.StringValue>, Tuple1<Integer>>>", ti.toString());
+		Assert.assertEquals("ObjectArrayTypeInfo<Java Tuple4<Double, ObjectArrayTypeInfo<GenericType<java.lang.Class>>, ValueType<org.apache.flink.types.StringValue>, Java Tuple1<Integer>>>", ti.toString());
 	}
 	
 	@Test

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java
new file mode 100644
index 0000000..500d8db
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.DoubleComparator;
+import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
+import org.apache.flink.api.common.typeutils.base.IntComparator;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+
+import org.apache.flink.api.java.typeutils.runtime.tuple.base.TuplePairComparatorTestBase;
+
+public class GenericPairComparatorTest extends TuplePairComparatorTestBase<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>> {
+
+	@SuppressWarnings("unchecked")
+	private Tuple3<Integer, String, Double>[] dataISD = new Tuple3[]{
+		new Tuple3<Integer, String, Double>(4, "hello", 20.0),
+		new Tuple3<Integer, String, Double>(4, "world", 23.2),
+		new Tuple3<Integer, String, Double>(5, "hello", 18.0),
+		new Tuple3<Integer, String, Double>(5, "world", 19.2),
+		new Tuple3<Integer, String, Double>(6, "hello", 16.0),
+		new Tuple3<Integer, String, Double>(6, "world", 17.2),
+		new Tuple3<Integer, String, Double>(7, "hello", 14.0),
+		new Tuple3<Integer, String, Double>(7, "world", 15.2)
+	};
+
+	@SuppressWarnings("unchecked")
+	private Tuple4<Integer, Float, Long, Double>[] dataIDL = new Tuple4[]{
+		new Tuple4<Integer, Float, Long, Double>(4, 0.11f, 14L, 20.0),
+		new Tuple4<Integer, Float, Long, Double>(4, 0.221f, 15L, 23.2),
+		new Tuple4<Integer, Float, Long, Double>(5, 0.33f, 15L, 18.0),
+		new Tuple4<Integer, Float, Long, Double>(5, 0.44f, 20L, 19.2),
+		new Tuple4<Integer, Float, Long, Double>(6, 0.55f, 20L, 16.0),
+		new Tuple4<Integer, Float, Long, Double>(6, 0.66f, 29L, 17.2),
+		new Tuple4<Integer, Float, Long, Double>(7, 0.77f, 29L, 14.0),
+		new Tuple4<Integer, Float, Long, Double>(7, 0.88f, 34L, 15.2)
+	};
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected GenericPairComparator<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>> createComparator(boolean ascending) {
+		int[] fields1 = new int[]{0, 2};
+		int[] fields2 = new int[]{0, 3};
+		TypeComparator[] comps1 = new TypeComparator[]{
+				new IntComparator(ascending),
+				new DoubleComparator(ascending)
+		};
+		TypeComparator[] comps2 = new TypeComparator[]{
+				new IntComparator(ascending),
+				new DoubleComparator(ascending)
+		};
+		TypeSerializer[] sers1 = new TypeSerializer[]{
+				IntSerializer.INSTANCE,
+				DoubleSerializer.INSTANCE
+		};
+		TypeSerializer[] sers2= new TypeSerializer[]{
+				IntSerializer.INSTANCE,
+				DoubleSerializer.INSTANCE
+		};
+		TypeComparator<Tuple3<Integer, String, Double>> comp1 = new TupleComparator<Tuple3<Integer, String, Double>>(fields1, comps1, sers1);
+		TypeComparator<Tuple4<Integer, Float, Long, Double>> comp2 = new TupleComparator<Tuple4<Integer, Float, Long, Double>>(fields2, comps2, sers2);
+		return new GenericPairComparator<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>>(comp1, comp2);
+	}
+
+	@Override
+	protected Tuple2<Tuple3<Integer, String, Double>[], Tuple4<Integer, Float, Long, Double>[]> getSortedTestData() {
+		return new Tuple2<Tuple3<Integer, String, Double>[], Tuple4<Integer, Float, Long, Double>[]>(dataISD, dataIDL);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparatorTest.java
deleted file mode 100644
index 0d9c29d..0000000
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparatorTest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.flink.api.common.typeutils.ComparatorTestBase;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
-import org.apache.flink.api.common.typeutils.base.IntComparator;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.runtime.TupleLeadingFieldComparator;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-
-public class TupleLeadingFieldComparatorTest extends ComparatorTestBase<Tuple3<Integer, String, Double>> {
-
-	@SuppressWarnings("unchecked")
-	Tuple3<Integer, String, Double>[] dataISD = new Tuple3[]{
-		new Tuple3<Integer, String, Double>(4, "hello", 20.0),
-		new Tuple3<Integer, String, Double>(5, "hello", 23.2),
-		new Tuple3<Integer, String, Double>(6, "world", 20.0),
-		new Tuple3<Integer, String, Double>(7, "hello", 20.0),
-		new Tuple3<Integer, String, Double>(8, "hello", 23.2),
-		new Tuple3<Integer, String, Double>(9, "world", 20.0),
-		new Tuple3<Integer, String, Double>(10, "hello", 20.0),
-		new Tuple3<Integer, String, Double>(11, "hello", 23.2)
-	};
-
-	@Override
-	protected TupleLeadingFieldComparator<Tuple3<Integer, String, Double>, Integer> createComparator(boolean ascending) {
-		return new TupleLeadingFieldComparator<Tuple3<Integer,String,Double>, Integer>(new IntComparator(ascending));
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	protected TupleSerializer<Tuple3<Integer, String, Double>> createSerializer() {
-		return new TupleSerializer<Tuple3<Integer, String, Double>>(
-				(Class<Tuple3<Integer, String, Double>>) (Class<?>) Tuple3.class,
-				new TypeSerializer[]{
-					new IntSerializer(),
-					new StringSerializer(),
-					new DoubleSerializer()});
-	}
-
-	@Override
-	protected Tuple3<Integer, String, Double>[] getSortedTestData() {
-		return dataISD;
-	}
-
-	@Override
-	protected void deepEquals(String message, Tuple3<Integer, String, Double> should, Tuple3<Integer, String, Double> is) {
-		for (int x = 0; x < should.getArity(); x++) {
-			assertEquals(should.getField(x), is.getField(x));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java
deleted file mode 100644
index 9847e76..0000000
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparatorTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.base.IntComparator;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.typeutils.runtime.TupleLeadingFieldPairComparator;
-
-import org.apache.flink.api.java.typeutils.runtime.tuple.base.TuplePairComparatorTestBase;
-
-public class TupleLeadingFieldPairComparatorTest extends TuplePairComparatorTestBase<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>> {
-
-	@SuppressWarnings("unchecked")
-	private Tuple3<Integer, String, Double>[] dataISD = new Tuple3[]{
-		new Tuple3<Integer, String, Double>(4, "hello", 20.0),
-		new Tuple3<Integer, String, Double>(5, "world", 23.2),
-		new Tuple3<Integer, String, Double>(6, "hello", 18.0),
-		new Tuple3<Integer, String, Double>(7, "world", 19.2),
-		new Tuple3<Integer, String, Double>(8, "hello", 16.0),
-		new Tuple3<Integer, String, Double>(9, "world", 17.2),
-		new Tuple3<Integer, String, Double>(10, "hello", 14.0),
-		new Tuple3<Integer, String, Double>(11, "world", 15.2)
-	};
-
-	@SuppressWarnings("unchecked")
-	private Tuple4<Integer, Float, Long, Double>[] dataIDL = new Tuple4[]{
-		new Tuple4<Integer, Float, Long, Double>(4, 0.11f, 14L, 20.0),
-		new Tuple4<Integer, Float, Long, Double>(5, 0.221f, 15L, 23.2),
-		new Tuple4<Integer, Float, Long, Double>(6, 0.33f, 15L, 18.0),
-		new Tuple4<Integer, Float, Long, Double>(7, 0.44f, 20L, 19.2),
-		new Tuple4<Integer, Float, Long, Double>(8, 0.55f, 20L, 16.0),
-		new Tuple4<Integer, Float, Long, Double>(9, 0.66f, 29L, 17.2),
-		new Tuple4<Integer, Float, Long, Double>(10, 0.77f, 29L, 14.0),
-		new Tuple4<Integer, Float, Long, Double>(11, 0.88f, 34L, 15.2)
-	};
-
-	@Override
-	protected TypePairComparator<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>> createComparator(boolean ascending) {
-		return new TupleLeadingFieldPairComparator<Integer, Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>>(
-				new IntComparator(ascending), new IntComparator(ascending));
-	}
-
-	@Override
-	protected Tuple2<Tuple3<Integer, String, Double>[], Tuple4<Integer, Float, Long, Double>[]> getSortedTestData() {
-		return new Tuple2<Tuple3<Integer, String, Double>[], Tuple4<Integer, Float, Long, Double>[]>(dataISD, dataIDL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparatorTest.java
deleted file mode 100644
index 1d7dccd..0000000
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparatorTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.base.DoubleComparator;
-import org.apache.flink.api.common.typeutils.base.IntComparator;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.typeutils.runtime.TuplePairComparator;
-
-import org.apache.flink.api.java.typeutils.runtime.tuple.base.TuplePairComparatorTestBase;
-
-public class TuplePairComparatorTest extends TuplePairComparatorTestBase<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>> {
-
-	@SuppressWarnings("unchecked")
-	private Tuple3<Integer, String, Double>[] dataISD = new Tuple3[]{
-		new Tuple3<Integer, String, Double>(4, "hello", 20.0),
-		new Tuple3<Integer, String, Double>(4, "world", 23.2),
-		new Tuple3<Integer, String, Double>(5, "hello", 18.0),
-		new Tuple3<Integer, String, Double>(5, "world", 19.2),
-		new Tuple3<Integer, String, Double>(6, "hello", 16.0),
-		new Tuple3<Integer, String, Double>(6, "world", 17.2),
-		new Tuple3<Integer, String, Double>(7, "hello", 14.0),
-		new Tuple3<Integer, String, Double>(7, "world", 15.2)
-	};
-
-	@SuppressWarnings("unchecked")
-	private Tuple4<Integer, Float, Long, Double>[] dataIDL = new Tuple4[]{
-		new Tuple4<Integer, Float, Long, Double>(4, 0.11f, 14L, 20.0),
-		new Tuple4<Integer, Float, Long, Double>(4, 0.221f, 15L, 23.2),
-		new Tuple4<Integer, Float, Long, Double>(5, 0.33f, 15L, 18.0),
-		new Tuple4<Integer, Float, Long, Double>(5, 0.44f, 20L, 19.2),
-		new Tuple4<Integer, Float, Long, Double>(6, 0.55f, 20L, 16.0),
-		new Tuple4<Integer, Float, Long, Double>(6, 0.66f, 29L, 17.2),
-		new Tuple4<Integer, Float, Long, Double>(7, 0.77f, 29L, 14.0),
-		new Tuple4<Integer, Float, Long, Double>(7, 0.88f, 34L, 15.2)
-	};
-
-	@SuppressWarnings("unchecked")
-	@Override
-	protected TuplePairComparator<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>> createComparator(boolean ascending) {
-		return new TuplePairComparator<Tuple3<Integer, String, Double>, Tuple4<Integer, Float, Long, Double>>(
-				new int[]{0, 2},
-				new int[]{0, 3},
-				new TypeComparator[]{
-					new IntComparator(ascending),
-					new DoubleComparator(ascending)
-				},
-				new TypeComparator[]{
-					new IntComparator(ascending),
-					new DoubleComparator(ascending)
-				}
-		);
-	}
-
-	@Override
-	protected Tuple2<Tuple3<Integer, String, Double>[], Tuple4<Integer, Float, Long, Double>[]> getSortedTestData() {
-		return new Tuple2<Tuple3<Integer, String, Double>[], Tuple4<Integer, Float, Long, Double>[]>(dataISD, dataIDL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
index 6db9035..34226a5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
@@ -316,7 +316,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T>
 		this.recordBufferForComparison.setReadPosition(pointer2);
 		
 		try {
-			return this.comparator.compare(this.recordBuffer, this.recordBufferForComparison);
+			return this.comparator.compareSerialized(this.recordBuffer, this.recordBufferForComparison);
 		} catch (IOException ioex) {
 			throw new RuntimeException("Error comparing two records.", ioex);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListComparator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListComparator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListComparator.java
index 53e596c..11064fa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListComparator.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListComparator.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.operators.testutils.types;
 import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.base.IntComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -31,6 +32,10 @@ public class IntListComparator extends TypeComparator<IntList> {
 	
 	private int reference;
 
+	private Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {new IntComparator(true)};
+
 	@Override
 	public int hash(IntList record) {
 		return record.getKey() * 73;
@@ -58,7 +63,7 @@ public class IntListComparator extends TypeComparator<IntList> {
 	}
 
 	@Override
-	public int compare(DataInputView source1, DataInputView source2) throws IOException {
+	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
 		return source1.readInt() - source2.readInt();
 	}
 
@@ -134,4 +139,15 @@ public class IntListComparator extends TypeComparator<IntList> {
 	public TypeComparator<IntList> duplicate() {
 		return new IntListComparator();
 	}
+
+	@Override
+	public Object[] extractKeys(IntList record) {
+		extractedKey[0] = record.getKey();
+		return extractedKey;
+	}
+
+	@Override public TypeComparator[] getComparators() {
+		return comparators;
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairComparator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairComparator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairComparator.java
index f5d31d6..53c76fb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairComparator.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairComparator.java
@@ -22,6 +22,7 @@ package org.apache.flink.runtime.operators.testutils.types;
 import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.base.IntComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -32,7 +33,11 @@ public class IntPairComparator extends TypeComparator<IntPair> {
 	private static final long serialVersionUID = 1L;
 	
 	private int reference;
-	
+
+	private final Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {new IntComparator(true)};
+
 	@Override
 	public int hash(IntPair object) {
 		return object.getKey() * 73;
@@ -60,7 +65,7 @@ public class IntPairComparator extends TypeComparator<IntPair> {
 	}
 	
 	@Override
-	public int compare(DataInputView source1, DataInputView source2) throws IOException {
+	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
 		return source1.readInt() - source2.readInt();
 	}
 
@@ -113,6 +118,15 @@ public class IntPairComparator extends TypeComparator<IntPair> {
 	}
 
 	@Override
+	public Object[] extractKeys(IntPair pair) {
+		extractedKey[0] = pair.getKey();
+		return extractedKey;
+	}
+	@Override public TypeComparator[] getComparators() {
+		return comparators;
+	}
+
+	@Override
 	public boolean supportsSerializationWithKeyNormalization() {
 		return true;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairComparator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairComparator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairComparator.java
index 4397ba5..66c87bb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairComparator.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairComparator.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.operators.testutils.types;
 import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.base.StringComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -32,6 +33,10 @@ public class StringPairComparator extends TypeComparator<StringPair> {
 	
 	private String reference;
 
+	private Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {new StringComparator(true)};
+
 	@Override
 	public int hash(StringPair record) {
 		return record.getKey().hashCode();
@@ -58,7 +63,7 @@ public class StringPairComparator extends TypeComparator<StringPair> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource)
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource)
 			throws IOException {
 		return StringValue.readString(firstSource).compareTo(StringValue.readString(secondSource));
 	}
@@ -110,4 +115,14 @@ public class StringPairComparator extends TypeComparator<StringPair> {
 	public TypeComparator<StringPair> duplicate() {
 		return new StringPairComparator();
 	}
+
+	@Override
+	public Object[] extractKeys(StringPair record) {
+		extractedKey[0] = record.getKey();
+		return extractedKey;
+	}
+
+	@Override public TypeComparator[] getComparators() {
+		return comparators;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/OutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/OutputEmitterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/OutputEmitterTest.java
index f109ca9..f55786c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/OutputEmitterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/OutputEmitterTest.java
@@ -27,6 +27,7 @@ import java.io.PipedOutputStream;
 
 import junit.framework.TestCase;
 
+import org.apache.flink.api.common.typeutils.base.IntComparator;
 import org.junit.Assert;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
@@ -381,6 +382,9 @@ public class OutputEmitterTest extends TestCase {
 	
 	@SuppressWarnings("serial")
 	private static class TestIntComparator extends TypeComparator<Integer> {
+		private final Comparable[] extractedKey = new Comparable[1];
+
+		private TypeComparator[] comparators = new TypeComparator[]{new IntComparator(true)};
 
 		@Override
 		public int hash(Integer record) {
@@ -402,7 +406,7 @@ public class OutputEmitterTest extends TestCase {
 		public int compare(Integer first, Integer second) { throw new UnsupportedOperationException(); }
 
 		@Override
-		public int compare(DataInputView firstSource, DataInputView secondSource) {
+		public int compareSerialized(DataInputView firstSource, DataInputView secondSource) {
 			throw new UnsupportedOperationException();
 		}
 
@@ -438,7 +442,17 @@ public class OutputEmitterTest extends TestCase {
 
 		@Override
 		public TypeComparator<Integer> duplicate() { throw new UnsupportedOperationException(); }
-		
+
+		@Override
+		public Object[] extractKeys(Integer record) {
+			extractedKey[0] = record;
+			return extractedKey;
+		}
+
+		@Override
+		public TypeComparator[] getComparators() {
+			return comparators;
+		}
 	}
 	
 //	@Test

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
index 0322024..5b32e12 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.iterative.nephele.customdanglingpagerank.types;
 import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.base.LongComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -30,7 +31,11 @@ public final class VertexWithAdjacencyListComparator extends TypeComparator<Vert
 	private static final long serialVersionUID = 1L;
 	
 	private long reference;
-	
+
+	private Comparable[] extractedKey = new Comparable[1];
+
+	private TypeComparator[] comparators = new TypeComparator[]{new LongComparator(true)};
+
 	@Override
 	public int hash(VertexWithAdjacencyList record) {
 		final long value = record.getVertexID();
@@ -61,7 +66,7 @@ public final class VertexWithAdjacencyListComparator extends TypeComparator<Vert
 	}
 
 	@Override
-	public int compare(DataInputView source1, DataInputView source2) throws IOException {
+	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
 		final long diff = source1.readLong() - source2.readLong();
 		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
 	}
@@ -129,4 +134,15 @@ public final class VertexWithAdjacencyListComparator extends TypeComparator<Vert
 	public VertexWithAdjacencyListComparator duplicate() {
 		return new VertexWithAdjacencyListComparator();
 	}
+
+	@Override
+	public Object[] extractKeys(VertexWithAdjacencyList record) {
+		extractedKey[0] = record.getVertexID();
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
index 5aa81ea..e46c36a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.iterative.nephele.customdanglingpagerank.types;
 import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.base.LongComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -30,7 +31,11 @@ public final class VertexWithRankAndDanglingComparator extends TypeComparator<Ve
 	private static final long serialVersionUID = 1L;
 	
 	private long reference;
-	
+
+	private Comparable[] extractedKey = new Comparable[1];
+
+	private TypeComparator[] comparators = new TypeComparator[]{new LongComparator(true)};
+
 	@Override
 	public int hash(VertexWithRankAndDangling record) {
 		final long value = record.getVertexID();
@@ -61,7 +66,7 @@ public final class VertexWithRankAndDanglingComparator extends TypeComparator<Ve
 	}
 
 	@Override
-	public int compare(DataInputView source1, DataInputView source2) throws IOException {
+	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
 		final long diff = source1.readLong() - source2.readLong();
 		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
 	}
@@ -134,4 +139,15 @@ public final class VertexWithRankAndDanglingComparator extends TypeComparator<Ve
 	public VertexWithRankAndDanglingComparator duplicate() {
 		return new VertexWithRankAndDanglingComparator();
 	}
+
+	@Override
+	public Object[] extractKeys(VertexWithRankAndDangling record) {
+		extractedKey[0] = record.getVertexID();
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
index 0d94c21..77a6a97 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.iterative.nephele.customdanglingpagerank.types;
 import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.base.LongComparator;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -30,7 +31,11 @@ public final class VertexWithRankComparator extends TypeComparator<VertexWithRan
 	private static final long serialVersionUID = 1L;
 	
 	private long reference;
-	
+
+	private Comparable[] extractedKey = new Comparable[1];
+
+	private TypeComparator[] comparators = new TypeComparator[]{new LongComparator(true)};
+
 	@Override
 	public int hash(VertexWithRank record) {
 		final long value = record.getVertexID();
@@ -61,7 +66,7 @@ public final class VertexWithRankComparator extends TypeComparator<VertexWithRan
 	}
 
 	@Override
-	public int compare(DataInputView source1, DataInputView source2) throws IOException {
+	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
 		final long diff = source1.readLong() - source2.readLong();
 		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
 	}
@@ -132,4 +137,15 @@ public final class VertexWithRankComparator extends TypeComparator<VertexWithRan
 	public VertexWithRankComparator duplicate() {
 		return new VertexWithRankComparator();
 	}
+
+	@Override
+	public Object[] extractKeys(VertexWithRank record) {
+		extractedKey[0] = record.getVertexID();
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
 }


[32/60] git commit: WebLog Analysis example

Posted by al...@apache.org.
WebLog Analysis example


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

Branch: refs/heads/master
Commit: a8dd9587fd649f368497f00b2b729a4b3ea9f504
Parents: 84b04be
Author: Kostas Tzoumas <ko...@kostass-mbp.fritz.box>
Authored: Tue Sep 9 16:45:01 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../java/relational/util/WebLogData.java        | 743 ++++++++++---------
 .../scala/relational/WebLogAnalysis.scala       | 348 +++++----
 2 files changed, 578 insertions(+), 513 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a8dd9587/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
index c36b617..3bd6c18 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
@@ -35,377 +35,394 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  */
 public class WebLogData {
 
+	public static final Object [][] DOCUMENTS = {
+			new Object [] {"url_0","dolor ad amet enim laoreet nostrud veniam aliquip ex nonummy diam dolore tincidunt tation exerci exerci wisi dolor nostrud "},
+			new Object [] {"url_1","wisi minim adipiscing nibh adipiscing ut nibh Lorem Ut nonummy euismod nibh wisi sit consectetuer exerci sed aliquip aliquip dolore aliquam enim dolore veniam aliquam euismod suscipit ad adipiscing exerci aliquip consectetuer euismod aliquip ad exerci ex nibh ex erat exerci laoreet lobortis quis "},
+			new Object [] {"url_2","diam sed convection aliquip amet commodo nonummy sed sed commodo commodo diam commodo adipiscing ad exerci magna exerci tation quis lobortis "},
+			new Object [] {"url_3","exerci suscipit sed lobortis amet lobortis aliquip nibh nostrud ad convection commodo ad nibh sed minim amet ad ea ea "},
+			new Object [] {"url_4","sit enim dolor quis laoreet ullamcorper veniam adipiscing ex quis commodo "},
+			new Object [] {"url_5","elit aliquip ea nisl oscillations sit dolor ipsum tincidunt ullamcorper dolore enim adipiscing laoreet elit ea volutpat adipiscing ea nibh nostrud Ut aliquam veniam Lorem laoreet veniam aliquip "},
+			new Object [] {"url_6","consectetuer ad sed suscipit euismod aliquip quis ullamcorper oscillations tation consectetuer tation amet suscipit nibh enim nonummy veniam commodo commodo diam euismod dolor Ut aliquip diam ex ad nonummy ad tincidunt minim exerci consectetuer veniam convection aliquam ut ut Lorem euismod sed ipsum volutpat "},
+			new Object [] {"url_7","Ut volutpat veniam ut consectetuer diam ut aliquam dolor nostrud erat consectetuer adipiscing exerci consectetuer Ut ullamcorper suscipit aliquam sed dolor nisl "},
+			new Object [] {"url_8","suscipit amet wisi nisl veniam lobortis sit Lorem aliquam nostrud aliquam ipsum ut laoreet suscipit Lorem laoreet editors adipiscing ullamcorper veniam erat consectetuer ut lobortis dolore elit sed tincidunt ipsum tation ullamcorper nonummy adipiscing ex ad laoreet ipsum suscipit lobortis lobortis Ut nonummy adipiscing erat volutpat aliquam "},
+			new Object [] {"url_9","nonummy commodo tation editors ut quis sit quis lobortis ea dolore oscillations diam ad dolor lobortis nisl ad veniam ullamcorper quis magna volutpat sit ipsum consectetuer dolore exerci commodo magna erat enim ut suscipit "},
+			new Object [] {"url_10","amet erat magna consectetuer tation tation aliquip nibh aliquam sed adipiscing ut commodo ex erat tincidunt aliquam ipsum Ut Ut sit tincidunt adipiscing suscipit minim sed erat dolor consectetuer Lorem consectetuer Lorem amet nibh diam ea ex enim suscipit wisi dolor nonummy magna enim euismod ullamcorper ut suscipit adipiscing "},
+			new Object [] {"url_11","ex quis exerci tation diam elit nostrud nostrud ut ipsum elit amet diam laoreet amet consectetuer volutpat sed lobortis "},
+			new Object [] {"url_12","elit suscipit sit ullamcorper ut ad erat ut dolor nostrud quis nisl enim erat dolor convection ad minim ut veniam nostrud sed editors adipiscing volutpat Ut aliquip commodo sed euismod adipiscing erat adipiscing dolore nostrud minim sed lobortis ea diam "},
+			new Object [] {"url_13","enim ut quis commodo veniam minim erat lobortis ad diam ex dolor tincidunt exerci ut aliquip tincidunt minim ut magna sed enim wisi veniam oscillations Lorem consectetuer "},
+			new Object [] {"url_14","nibh ipsum ullamcorper volutpat ut wisi dolor quis amet euismod quis ipsum ipsum minim tation volutpat sit exerci volutpat amet nonummy euismod veniam consectetuer sit consectetuer tincidunt nibh aliquam lobortis tation veniam ut ullamcorper wisi magna Ut volutpat consectetuer erat quis dolore ea tation "},
+			new Object [] {"url_15","ad wisi sed enim aliquam oscillations nibh Lorem lobortis veniam nibh laoreet nonummy sed nibh Lorem adipiscing diam magna nostrud magna oscillations ut oscillations elit nostrud diam editors Lorem "},
+			new Object [] {"url_16","nostrud volutpat veniam exerci tincidunt nostrud quis elit ipsum ea nonummy volutpat dolor elit lobortis magna nisl ut ullamcorper magna Lorem exerci nibh nisl magna editors erat aliquam aliquam ullamcorper sit aliquam sit nostrud oscillations consectetuer adipiscing suscipit convection exerci ea ullamcorper ex nisl "},
+			new Object [] {"url_17","ad ex aliquam erat aliquam elit veniam laoreet ut amet amet nostrud ut adipiscing Ut Lorem suscipit ex magna ullamcorper aliquam ullamcorper ullamcorper amet amet commodo aliquam volutpat nonummy nonummy tincidunt amet tation tincidunt volutpat ut veniam nisl erat dolor enim nonummy nostrud adipiscing laoreet adipiscing "},
+			new Object [] {"url_18","lobortis ipsum ex tincidunt tincidunt editors euismod consectetuer ipsum adipiscing lobortis exerci adipiscing nonummy nisl dolore nonummy erat exerci nisl ut dolore wisi volutpat lobortis magna "},
+			new Object [] {"url_19","ipsum tation laoreet tation adipiscing wisi nibh diam Ut suscipit ad wisi "},
+			new Object [] {"url_20","diam Lorem enim wisi ad lobortis dolor Ut ipsum amet dolore consectetuer nisl exerci nisl nonummy minim Ut erat oscillations ut Lorem nostrud dolore Ut dolore exerci ad ipsum dolore ex dolore aliquip sed aliquam ex aliquip magna amet ex dolore oscillations aliquip tation magna Ut "},
+			new Object [] {"url_21","lobortis ut amet ex nisl ullamcorper tincidunt ut elit diam quis suscipit ad amet ipsum magna Ut ex tincidunt "},
+			new Object [] {"url_22","amet commodo nisl ad quis lobortis ut commodo sit ut erat exerci lobortis suscipit nibh ut nostrud ut adipiscing commodo commodo quis quis nostrud nisl ipsum nostrud laoreet Lorem nostrud erat nostrud amet consectetuer laoreet oscillations wisi sit magna nibh amet "},
+			new Object [] {"url_23","adipiscing suscipit suscipit aliquip suscipit consectetuer minim magna ea erat nibh sit suscipit sed dolor oscillations nonummy volutpat ut tincidunt "},
+			new Object [] {"url_24","commodo sed tincidunt aliquip aliquip dolore commodo nonummy sed erat ut ex exerci dolore adipiscing tincidunt ex diam amet aliquam "},
+			new Object [] {"url_25","consectetuer consectetuer exerci quis ea veniam aliquam laoreet minim ex "},
+			new Object [] {"url_26","dolor exerci euismod minim magna quis erat consectetuer sed ex erat dolore quis ut oscillations ullamcorper Lorem exerci ex nibh ut exerci ullamcorper veniam nibh ut commodo ut Ut nostrud tincidunt tincidunt ad dolore Lorem ea tation enim erat nibh ut ea nonummy sed sed wisi nisl dolore "},
+			new Object [] {"url_27","amet elit ea ea nostrud editors Ut nostrud amet laoreet adipiscing ut nisl nonummy tincidunt ea ipsum ex dolore dolore oscillations sit minim Ut wisi ut laoreet minim elit "},
+			new Object [] {"url_28","wisi exerci volutpat Ut nostrud euismod minim Ut sit euismod ut ea magna consectetuer nisl ad minim tation nisl adipiscing Lorem aliquam quis exerci erat minim aliquip sit Lorem wisi wisi ut "},
+			new Object [] {"url_29","amet sed laoreet amet aliquam minim enim tincidunt Lorem sit aliquip amet suscipit ut laoreet elit suscipit erat ut tincidunt suscipit ipsum sed euismod elit dolore euismod dolore ut dolor nostrud ipsum tincidunt commodo adipiscing aliquam ut wisi dolor dolor suscipit "},
+			new Object [] {"url_30","euismod Lorem ex tincidunt amet enim minim suscipit exerci diam veniam amet nostrud ea ea "},
+			new Object [] {"url_31","ex ipsum sit euismod euismod ullamcorper tincidunt ut wisi ea adipiscing sed diam tation ipsum dolor aliquam veniam nonummy aliquip aliquip Lorem ut minim nisl tation sit exerci ullamcorper Ut dolor euismod aliquam consectetuer ad nonummy commodo exerci "},
+			new Object [] {"url_32","volutpat ipsum lobortis nisl veniam minim adipiscing dolor editors quis nostrud amet nostrud "},
+			new Object [] {"url_33","commodo wisi aliquip ut aliquam sed nostrud ex diam ad nostrud enim ut amet enim ea ad sed tation nostrud suscipit ea magna magna Lorem amet lobortis ut quis nibh aliquam aliquam exerci aliquip lobortis consectetuer enim wisi ea nisl laoreet erat dolore "},
+			new Object [] {"url_34","tincidunt adipiscing enim tation nibh Ut dolore tincidunt tation laoreet suscipit minim aliquam volutpat laoreet suscipit tincidunt nibh ut ut sit nostrud nonummy tincidunt exerci sit ad sed consectetuer minim dolor dolore laoreet nostrud nibh laoreet ea adipiscing exerci dolore ipsum "},
+			new Object [] {"url_35","tation ut erat ut tation dolor Lorem laoreet Lorem elit adipiscing wisi aliquip nostrud elit Ut volutpat ea aliquam aliquip "},
+			new Object [] {"url_36","lobortis enim ullamcorper adipiscing consectetuer aliquip wisi enim minim Ut minim elit elit aliquam exerci ullamcorper amet lobortis adipiscing diam laoreet consectetuer nostrud diam diam amet ut enim ullamcorper aliquip diam ut nostrud diam magna amet nonummy commodo wisi enim ullamcorper suscipit euismod dolore tincidunt magna suscipit elit "},
+			new Object [] {"url_37","elit adipiscing nisl nisl ex aliquip nibh sed ut ad Lorem elit consectetuer ad volutpat lobortis amet veniam ipsum nibh ut consectetuer editors ad aliquam "},
+			new Object [] {"url_38","elit quis nibh adipiscing sit consectetuer ut euismod quis tincidunt quis nisl consectetuer dolor diam suscipit quis dolore Lorem suscipit nonummy sed ex "},
+			new Object [] {"url_39","nisl sit consectetuer elit oscillations enim ipsum enim nostrud adipiscing nostrud editors aliquam "},
+			new Object [] {"url_40","sed wisi dolor diam commodo ullamcorper commodo nostrud ullamcorper laoreet minim dolore suscipit laoreet tation aliquip "},
+			new Object [] {"url_41","ad consectetuer exerci nisl exerci amet enim diam lobortis Lorem ex volutpat volutpat nibh aliquam ut ullamcorper volutpat nostrud ut adipiscing ullamcorper "},
+			new Object [] {"url_42","minim laoreet tation magna veniam ut ea sit ipsum tincidunt Ut amet ex aliquip ex euismod exerci wisi elit editors ad amet veniam ad editors "},
+			new Object [] {"url_43","ut nisl ad ullamcorper nibh Ut editors exerci enim exerci ea laoreet veniam ea amet exerci volutpat amet ad "},
+			new Object [] {"url_44","volutpat tincidunt enim amet sed tincidunt consectetuer ullamcorper nisl Ut adipiscing tation ad ad amet nonummy elit erat nibh Lorem erat elit laoreet consectetuer sed aliquip nostrud "},
+			new Object [] {"url_45","sed aliquam ut ut consectetuer wisi euismod enim erat euismod quis exerci amet tation sit "},
+			new Object [] {"url_46","lobortis oscillations tation aliquam dolore Lorem aliquip tation exerci ullamcorper aliquam aliquip lobortis ex tation dolor ut ut sed suscipit nisl ullamcorper sed editors laoreet aliquip enim dolor veniam tincidunt sed euismod tation "},
+			new Object [] {"url_47","Lorem Lorem ut wisi ad ut tation consectetuer exerci convection tation ullamcorper sed dolore quis aliquam ipsum lobortis commodo nonummy "},
+			new Object [] {"url_48","laoreet minim veniam nisl elit sit amet commodo ex ullamcorper suscipit aliquip laoreet convection Ut ex minim aliquam "},
+			new Object [] {"url_49","lobortis nonummy minim amet sit veniam quis consectetuer tincidunt laoreet quis "},
+			new Object [] {"url_50","lobortis nisl commodo dolor amet nibh editors enim magna minim elit euismod diam laoreet laoreet ad minim sed ut Ut lobortis adipiscing quis sed ut aliquam oscillations exerci tation consectetuer lobortis elit tincidunt consectetuer minim amet dolore quis aliquam Ut exerci sed aliquam quis quis ullamcorper Ut ex tincidunt "},
+			new Object [] {"url_51","nostrud nisl ea erat ut suscipit Ut sit oscillations ullamcorper nonummy magna lobortis dolore editors tincidunt nostrud suscipit ex quis tation ut sit amet nostrud laoreet ex tincidunt "},
+			new Object [] {"url_52","ea tation commodo elit sed ex sed quis enim nisl magna laoreet adipiscing amet sit nostrud consectetuer nibh tincidunt veniam ex veniam euismod exerci sed dolore suscipit nisl tincidunt euismod quis Ut enim euismod dolor diam exerci magna exerci ut exerci nisl "},
+			new Object [] {"url_53","volutpat amet Ut lobortis dolor tation minim nonummy lobortis convection nostrud "},
+			new Object [] {"url_54","ullamcorper commodo Ut amet sit nostrud aliquam ad amet wisi enim nostrud ipsum nisl veniam erat aliquam ex aliquam dolor dolor ut consectetuer euismod exerci elit exerci Ut ea minim enim consectetuer ad consectetuer nonummy convection adipiscing ad ullamcorper lobortis nonummy laoreet nonummy aliquam ullamcorper ad nostrud amet "},
+			new Object [] {"url_55","wisi magna editors amet aliquam diam amet aliquip nisl consectetuer laoreet nonummy suscipit euismod diam enim tation elit ut lobortis quis euismod suscipit nostrud ea ea commodo lobortis dolore Ut nisl nostrud dolor laoreet euismod ea dolore aliquam ut Lorem exerci ex sit "},
+			new Object [] {"url_56","ex dolor veniam wisi laoreet ut exerci diam ad ex ut ut laoreet ut nisl ullamcorper nisl "},
+			new Object [] {"url_57","diam adipiscing Ut ut Lorem amet erat elit erat magna adipiscing euismod elit ullamcorper nostrud aliquam dolor ullamcorper sit tation tation "},
+			new Object [] {"url_58","laoreet convection veniam lobortis dolore ut nonummy commodo erat lobortis veniam nostrud dolore minim commodo ut consectetuer magna erat ea dolore Lorem suscipit ex ipsum exerci sed enim ea tation suscipit enim adipiscing "},
+			new Object [] {"url_59","amet ut ut Ut ad dolor quis ad magna exerci suscipit magna nibh commodo euismod amet euismod wisi diam suscipit dolore Lorem dolor ex amet exerci aliquip ut ut lobortis quis elit minim sed Lorem "},
+			new Object [] {"url_60","ut ut amet ullamcorper amet euismod dolor amet elit exerci adipiscing sed suscipit sed exerci wisi diam veniam wisi suscipit ut quis nibh ullamcorper ex quis magna dolore volutpat editors minim ut sit aliquip oscillations nisl ipsum "},
+			new Object [] {"url_61","nibh nostrud tincidunt lobortis adipiscing adipiscing ullamcorper ullamcorper ipsum nisl ullamcorper aliquip laoreet commodo ut tation wisi diam commodo aliquip commodo suscipit tincidunt volutpat elit enim laoreet ut nostrud ad nonummy ipsum "},
+			new Object [] {"url_62","Ut ut minim enim amet euismod erat elit commodo consectetuer Ut quis dolor ex diam quis wisi tation tincidunt laoreet volutpat "},
+			new Object [] {"url_63","ut erat volutpat euismod amet ea nonummy lobortis ut Ut ea veniam sed veniam nostrud "},
+			new Object [] {"url_64","tation dolor suscipit minim nisl wisi consectetuer aliquip tation Ut commodo ut dolore consectetuer elit wisi nisl ipsum "},
+			new Object [] {"url_65","ullamcorper nisl Lorem magna tation veniam aliquam diam amet euismod "},
+			new Object [] {"url_66","euismod aliquam tincidunt Ut volutpat ea lobortis sit ut volutpat ut lobortis ut lobortis ut nisl amet dolor sed ipsum enim ullamcorper diam euismod nostrud wisi erat quis diam nibh Ut dolore sed amet tation enim diam "},
+			new Object [] {"url_67","amet minim minim amet laoreet Lorem aliquam veniam elit volutpat magna adipiscing enim enim euismod laoreet sed ex sed aliquam ad ea ut adipiscing suscipit ex minim dolore minim ea laoreet nisl "},
+			new Object [] {"url_68","aliquam ea volutpat ut wisi tation tation nibh nisl erat laoreet ea volutpat dolor dolor aliquam exerci quis ullamcorper aliquam ut quis suscipit "},
+			new Object [] {"url_69","quis exerci ut aliquip wisi dolore magna nibh consectetuer magna tation ullamcorper lobortis sed amet adipiscing minim suscipit nibh nibh nostrud euismod enim "},
+			new Object [] {"url_70","tation enim consectetuer adipiscing wisi laoreet diam aliquip nostrud elit nostrud aliquip ea minim amet diam dolore "},
+			new Object [] {"url_71","consectetuer tincidunt nibh amet tation nonummy sit tation diam sed diam tation "},
+			new Object [] {"url_72","Lorem ut nostrud nonummy minim quis euismod lobortis nostrud nonummy adipiscing tincidunt consectetuer ut nibh ad suscipit dolor ut elit dolore amet ut quis tation ullamcorper nonummy laoreet ullamcorper aliquam dolore convection dolor tincidunt ut ullamcorper ex dolor suscipit erat oscillations ad "},
+			new Object [] {"url_73","elit Ut commodo ut ullamcorper ullamcorper ut euismod commodo diam aliquip suscipit consectetuer exerci tation nostrud ut wisi exerci sed ut elit sed volutpat Lorem nibh laoreet consectetuer ex Lorem elit aliquam commodo lobortis ad "},
+			new Object [] {"url_74","quis magna laoreet commodo aliquam nisl ullamcorper veniam tation wisi consectetuer commodo consectetuer ad dolore aliquam dolor elit amet sit amet nibh commodo erat veniam aliquip dolore ad magna ad ipsum Ut exerci ea volutpat nisl amet nostrud sit "},
+			new Object [] {"url_75","tincidunt suscipit sit aliquip aliquam adipiscing dolore exerci Ut suscipit ut sit laoreet suscipit wisi sit enim nonummy consectetuer dolore editors "},
+			new Object [] {"url_76","veniam ullamcorper tation sit suscipit dolor suscipit veniam sit Lorem quis sed nostrud ad tincidunt elit adipiscing "},
+			new Object [] {"url_77","volutpat sit amet veniam quis ipsum nibh elit enim commodo magna veniam magna convection "},
+			new Object [] {"url_78","tation dolore minim elit nisl volutpat tation laoreet enim nostrud exerci dolore tincidunt aliquip Lorem ipsum nostrud quis adipiscing ullamcorper erat lobortis tation commodo Ut ipsum commodo magna ad ipsum ut enim "},
+			new Object [] {"url_79","lobortis amet elit Lorem amet nonummy commodo tation ex ea amet Lorem ea nonummy commodo veniam volutpat nibh wisi ad ipsum euismod ea convection nostrud nisl erat veniam Ut aliquip ad aliquip editors wisi magna tation nostrud nonummy adipiscing ullamcorper aliquip "},
+			new Object [] {"url_80","tincidunt nostrud nostrud magna ea euismod ea consectetuer nisl exerci ea dolor nisl commodo ex erat ipsum exerci suscipit ad nisl ea nonummy suscipit adipiscing laoreet sit euismod nibh adipiscing sed minim commodo amet "},
+			new Object [] {"url_81","nostrud erat ut sed editors erat amet magna lobortis diam laoreet dolor amet nibh ut ipsum ipsum amet ut sed ut exerci elit suscipit wisi magna ut veniam nisl commodo enim adipiscing laoreet ad Lorem oscillations "},
+			new Object [] {"url_82","quis commodo nibh nibh volutpat suscipit dolore magna tincidunt nibh ut ad ullamcorper ullamcorper quis enim ad ut tation minim laoreet veniam dolor sed tincidunt exerci exerci nostrud ullamcorper amet ut ut ullamcorper "},
+			new Object [] {"url_83","sit suscipit volutpat elit tation elit sed sed dolor ex ex ipsum euismod laoreet magna lobortis ad "},
+			new Object [] {"url_84","lobortis ipsum euismod enim ea tation veniam tation oscillations aliquip consectetuer euismod ut sed lobortis tation oscillations commodo euismod laoreet suscipit amet elit ullamcorper volutpat aliquam ea enim ullamcorper consectetuer laoreet tation quis ut commodo erat euismod dolor laoreet ullamcorper laoreet "},
+			new Object [] {"url_85","adipiscing sit quis commodo consectetuer quis enim euismod exerci nonummy ea nostrud Ut veniam sit aliquip nisl enim "},
+			new Object [] {"url_86","nostrud dolore veniam veniam wisi aliquip adipiscing diam sed quis ullamcorper "},
+			new Object [] {"url_87","quis Lorem suscipit Ut nibh diam euismod consectetuer lobortis ipsum sed suscipit consectetuer euismod laoreet ut wisi nisl elit quis commodo adipiscing adipiscing suscipit aliquam nisl quis magna ipsum enim ad quis ea magna Lorem nibh ea "},
+			new Object [] {"url_88","euismod commodo sed tincidunt Ut veniam consectetuer quis erat ex ea erat laoreet commodo nibh minim "},
+			new Object [] {"url_89","tation diam editors Ut enim nibh Lorem volutpat quis diam suscipit exerci wisi ad "},
+			new Object [] {"url_90","volutpat editors ea nibh wisi ad amet volutpat nisl ullamcorper nibh volutpat minim ex ut sit veniam Lorem consectetuer quis ad sit suscipit volutpat wisi diam sed tincidunt ipsum minim convection ea diam oscillations quis lobortis "},
+			new Object [] {"url_91","enim minim nonummy ea minim euismod adipiscing editors volutpat magna sit magna ut ipsum ut "},
+			new Object [] {"url_92","nisl Ut commodo amet euismod lobortis ea ea wisi commodo Lorem sit ipsum volutpat nonummy exerci erat elit exerci magna ad erat enim laoreet quis nostrud wisi ut veniam amet ullamcorper lobortis ad suscipit volutpat veniam nostrud nibh quis ipsum dolore consectetuer veniam ipsum aliquip dolore sed laoreet ipsum "},
+			new Object [] {"url_93","nonummy aliquam ad lobortis Lorem erat ad tation Lorem exerci ex "},
+			new Object [] {"url_94","nonummy dolore commodo exerci ex quis ut suscipit elit laoreet sit tation magna veniam ea sit nonummy veniam Lorem quis nibh aliquip exerci amet ullamcorper adipiscing erat nisl editors diam commodo ad euismod adipiscing ea suscipit exerci aliquip volutpat tation enim volutpat sit "},
+			new Object [] {"url_95","sit suscipit oscillations ipsum nibh dolor ea dolore ea elit ipsum minim editors magna consectetuer ullamcorper commodo nonummy sit nostrud aliquip sit erat ullamcorper ullamcorper nibh veniam erat quis dolore nonummy "},
+			new Object [] {"url_96","nostrud quis ut volutpat magna ad quis adipiscing Lorem commodo exerci laoreet magna adipiscing erat quis wisi ea ea laoreet enim convection ad dolor nisl amet nibh aliquam adipiscing tincidunt minim diam Lorem commodo adipiscing volutpat "},
+			new Object [] {"url_97","laoreet laoreet suscipit nostrud dolore adipiscing volutpat Ut sed nisl diam ullamcorper ex ut ut dolor amet nostrud euismod dolore veniam veniam enim tation veniam ea minim minim volutpat tincidunt "},
+			new Object [] {"url_98","quis lobortis amet wisi nostrud ipsum aliquam convection tincidunt dolore ullamcorper nibh lobortis volutpat ea nostrud oscillations minim nonummy enim ad lobortis exerci ipsum ullamcorper nibh nonummy diam amet enim veniam ut nostrud "},
+			new Object [] {"url_99","aliquam wisi suscipit commodo diam amet amet magna nisl enim nostrud tation nisl nostrud nibh ut "}
+	};
+
+	public static final Object [][] RANKS = {
+			new Object [] {30,"url_0",43},
+			new Object [] {82,"url_1",39},
+			new Object [] {56,"url_2",31},
+			new Object [] {96,"url_3",36},
+			new Object [] {31,"url_4",36},
+			new Object [] {29,"url_5",6},
+			new Object [] {33,"url_6",48},
+			new Object [] {66,"url_7",40},
+			new Object [] {28,"url_8",51},
+			new Object [] {9,"url_9",4},
+			new Object [] {49,"url_10",24},
+			new Object [] {26,"url_11",12},
+			new Object [] {39,"url_12",46},
+			new Object [] {84,"url_13",53},
+			new Object [] {29,"url_14",50},
+			new Object [] {21,"url_15",12},
+			new Object [] {69,"url_16",34},
+			new Object [] {11,"url_17",38},
+			new Object [] {96,"url_18",13},
+			new Object [] {56,"url_19",48},
+			new Object [] {18,"url_20",36},
+			new Object [] {31,"url_21",21},
+			new Object [] {29,"url_22",11},
+			new Object [] {71,"url_23",30},
+			new Object [] {85,"url_24",48},
+			new Object [] {19,"url_25",45},
+			new Object [] {69,"url_26",9},
+			new Object [] {20,"url_27",51},
+			new Object [] {33,"url_28",46},
+			new Object [] {75,"url_29",38},
+			new Object [] {96,"url_30",51},
+			new Object [] {73,"url_31",40},
+			new Object [] {67,"url_32",16},
+			new Object [] {24,"url_33",24},
+			new Object [] {27,"url_34",35},
+			new Object [] {33,"url_35",35},
+			new Object [] {7,"url_36",22},
+			new Object [] {83,"url_37",41},
+			new Object [] {23,"url_38",49},
+			new Object [] {41,"url_39",33},
+			new Object [] {66,"url_40",38},
+			new Object [] {4,"url_41",52},
+			new Object [] {34,"url_42",4},
+			new Object [] {28,"url_43",12},
+			new Object [] {14,"url_44",14},
+			new Object [] {41,"url_45",11},
+			new Object [] {48,"url_46",37},
+			new Object [] {75,"url_47",41},
+			new Object [] {78,"url_48",3},
+			new Object [] {63,"url_49",28}
+	};
+
+
+	public static final Object [][] VISITS = {
+			new Object [] {"url_2","2003-12-17"},
+			new Object [] {"url_9","2008-11-11"},
+			new Object [] {"url_14","2003-11-5"},
+			new Object [] {"url_46","2009-2-16"},
+			new Object [] {"url_14","2004-11-9"},
+			new Object [] {"url_36","2001-3-9"},
+			new Object [] {"url_35","2006-8-13"},
+			new Object [] {"url_22","2008-1-18"},
+			new Object [] {"url_36","2002-3-9"},
+			new Object [] {"url_13","2007-7-17"},
+			new Object [] {"url_23","2009-6-16"},
+			new Object [] {"url_16","2000-7-15"},
+			new Object [] {"url_41","2002-5-10"},
+			new Object [] {"url_6","2004-11-9"},
+			new Object [] {"url_5","2003-6-7"},
+			new Object [] {"url_22","2002-11-5"},
+			new Object [] {"url_11","2007-7-21"},
+			new Object [] {"url_38","2009-12-2"},
+			new Object [] {"url_6","2004-11-2"},
+			new Object [] {"url_46","2000-6-4"},
+			new Object [] {"url_34","2003-9-2"},
+			new Object [] {"url_31","2008-2-24"},
+			new Object [] {"url_0","2003-2-2"},
+			new Object [] {"url_47","2003-7-8"},
+			new Object [] {"url_49","2009-9-13"},
+			new Object [] {"url_11","2003-4-2"},
+			new Object [] {"url_20","2000-6-18"},
+			new Object [] {"url_38","2000-2-22"},
+			new Object [] {"url_44","2009-2-17"},
+			new Object [] {"url_26","2000-6-21"},
+			new Object [] {"url_13","2000-11-25"},
+			new Object [] {"url_47","2005-4-19"},
+			new Object [] {"url_46","2008-1-7"},
+			new Object [] {"url_33","2004-12-24"},
+			new Object [] {"url_32","2009-2-8"},
+			new Object [] {"url_26","2000-9-21"},
+			new Object [] {"url_9","2002-8-18"},
+			new Object [] {"url_38","2002-11-27"},
+			new Object [] {"url_37","2008-2-26"},
+			new Object [] {"url_1","2007-3-22"},
+			new Object [] {"url_37","2002-3-20"},
+			new Object [] {"url_27","2008-11-12"},
+			new Object [] {"url_30","2000-12-16"},
+			new Object [] {"url_48","2000-12-17"},
+			new Object [] {"url_46","2008-4-16"},
+			new Object [] {"url_29","2006-3-9"},
+			new Object [] {"url_0","2007-7-26"},
+			new Object [] {"url_46","2009-12-15"},
+			new Object [] {"url_34","2002-2-13"},
+			new Object [] {"url_24","2009-3-1"},
+			new Object [] {"url_43","2007-11-4"},
+			new Object [] {"url_3","2004-2-16"},
+			new Object [] {"url_26","2000-10-26"},
+			new Object [] {"url_42","2004-7-14"},
+			new Object [] {"url_13","2004-9-10"},
+			new Object [] {"url_21","2000-2-21"},
+			new Object [] {"url_9","2006-6-5"},
+			new Object [] {"url_46","2001-12-17"},
+			new Object [] {"url_24","2006-12-8"},
+			new Object [] {"url_25","2006-9-2"},
+			new Object [] {"url_37","2002-6-26"},
+			new Object [] {"url_18","2006-6-2"},
+			new Object [] {"url_46","2003-5-24"},
+			new Object [] {"url_32","2000-10-17"},
+			new Object [] {"url_45","2002-1-12"},
+			new Object [] {"url_12","2005-12-13"},
+			new Object [] {"url_49","2009-3-9"},
+			new Object [] {"url_31","2001-9-19"},
+			new Object [] {"url_22","2002-7-9"},
+			new Object [] {"url_27","2005-2-3"},
+			new Object [] {"url_43","2008-7-15"},
+			new Object [] {"url_20","2000-3-23"},
+			new Object [] {"url_25","2002-5-8"},
+			new Object [] {"url_41","2004-4-27"},
+			new Object [] {"url_17","2008-7-17"},
+			new Object [] {"url_26","2009-12-16"},
+			new Object [] {"url_34","2006-2-10"},
+			new Object [] {"url_8","2009-4-14"},
+			new Object [] {"url_16","2000-2-24"},
+			new Object [] {"url_2","2009-2-10"},
+			new Object [] {"url_35","2003-2-24"},
+			new Object [] {"url_34","2008-3-16"},
+			new Object [] {"url_27","2005-1-5"},
+			new Object [] {"url_8","2008-12-10"},
+			new Object [] {"url_38","2009-2-11"},
+			new Object [] {"url_38","2006-11-3"},
+			new Object [] {"url_47","2003-2-13"},
+			new Object [] {"url_8","2008-11-17"},
+			new Object [] {"url_26","2009-5-11"},
+			new Object [] {"url_12","2007-11-26"},
+			new Object [] {"url_10","2003-1-13"},
+			new Object [] {"url_8","2005-9-23"},
+			new Object [] {"url_42","2001-4-5"},
+			new Object [] {"url_30","2009-12-10"},
+			new Object [] {"url_2","2003-1-3"},
+			new Object [] {"url_2","2009-2-19"},
+			new Object [] {"url_7","2000-6-25"},
+			new Object [] {"url_15","2004-9-26"},
+			new Object [] {"url_25","2009-10-5"},
+			new Object [] {"url_23","2009-8-9"},
+			new Object [] {"url_27","2004-4-3"},
+			new Object [] {"url_37","2008-6-9"},
+			new Object [] {"url_9","2002-5-25"},
+			new Object [] {"url_43","2009-5-18"},
+			new Object [] {"url_21","2008-4-19"},
+			new Object [] {"url_12","2001-12-25"},
+			new Object [] {"url_16","2006-9-25"},
+			new Object [] {"url_27","2002-1-2"},
+			new Object [] {"url_2","2009-1-21"},
+			new Object [] {"url_31","2009-3-20"},
+			new Object [] {"url_42","2002-3-1"},
+			new Object [] {"url_31","2001-11-26"},
+			new Object [] {"url_20","2003-5-15"},
+			new Object [] {"url_32","2004-1-22"},
+			new Object [] {"url_28","2008-9-16"},
+			new Object [] {"url_27","2006-7-3"},
+			new Object [] {"url_11","2008-12-26"},
+			new Object [] {"url_15","2004-8-16"},
+			new Object [] {"url_34","2002-10-5"},
+			new Object [] {"url_44","2000-2-15"},
+			new Object [] {"url_9","2000-10-23"},
+			new Object [] {"url_45","2005-4-24"},
+			new Object [] {"url_0","2006-8-7"},
+			new Object [] {"url_48","2003-8-7"},
+			new Object [] {"url_8","2007-12-13"},
+			new Object [] {"url_42","2003-8-2"},
+			new Object [] {"url_25","2008-3-5"},
+			new Object [] {"url_3","2007-3-9"},
+			new Object [] {"url_49","2003-10-7"},
+			new Object [] {"url_18","2007-12-6"},
+			new Object [] {"url_3","2006-7-5"},
+			new Object [] {"url_27","2000-9-14"},
+			new Object [] {"url_42","2002-10-20"},
+			new Object [] {"url_44","2007-1-13"},
+			new Object [] {"url_6","2003-1-21"},
+			new Object [] {"url_40","2009-10-20"},
+			new Object [] {"url_28","2009-6-17"},
+			new Object [] {"url_22","2000-2-17"},
+			new Object [] {"url_3","2005-1-15"},
+			new Object [] {"url_9","2008-12-9"},
+			new Object [] {"url_9","2005-2-19"},
+			new Object [] {"url_28","2000-4-22"},
+			new Object [] {"url_44","2001-9-9"},
+			new Object [] {"url_43","2008-6-21"},
+			new Object [] {"url_39","2008-5-9"},
+			new Object [] {"url_15","2006-9-15"},
+			new Object [] {"url_23","2001-12-18"},
+			new Object [] {"url_14","2002-5-23"},
+			new Object [] {"url_11","2007-7-11"},
+			new Object [] {"url_34","2000-12-8"},
+			new Object [] {"url_47","2005-7-3"},
+			new Object [] {"url_38","2004-3-26"},
+			new Object [] {"url_19","2003-9-14"},
+			new Object [] {"url_24","2007-7-16"},
+			new Object [] {"url_40","2008-8-21"},
+			new Object [] {"url_17","2007-12-4"},
+			new Object [] {"url_25","2006-6-24"},
+			new Object [] {"url_2","2000-10-8"},
+			new Object [] {"url_12","2008-6-10"},
+			new Object [] {"url_11","2004-11-24"},
+			new Object [] {"url_13","2005-11-3"},
+			new Object [] {"url_43","2005-1-2"},
+			new Object [] {"url_14","2008-6-12"},
+			new Object [] {"url_43","2001-8-27"},
+			new Object [] {"url_45","2000-3-3"},
+			new Object [] {"url_0","2006-9-27"},
+			new Object [] {"url_22","2007-12-18"},
+			new Object [] {"url_25","2006-4-4"},
+			new Object [] {"url_32","2001-6-25"},
+			new Object [] {"url_6","2007-6-9"},
+			new Object [] {"url_8","2009-10-3"},
+			new Object [] {"url_15","2003-2-23"},
+			new Object [] {"url_37","2000-5-6"},
+			new Object [] {"url_27","2004-3-21"},
+			new Object [] {"url_17","2005-6-20"},
+			new Object [] {"url_2","2004-2-27"},
+			new Object [] {"url_36","2005-3-16"},
+			new Object [] {"url_1","2009-12-3"},
+			new Object [] {"url_9","2004-4-27"},
+			new Object [] {"url_18","2009-5-26"},
+			new Object [] {"url_31","2000-9-21"},
+			new Object [] {"url_12","2008-9-25"},
+			new Object [] {"url_2","2004-2-16"},
+			new Object [] {"url_28","2008-11-12"},
+			new Object [] {"url_28","2001-6-26"},
+			new Object [] {"url_12","2006-3-15"},
+			new Object [] {"url_0","2009-3-1"},
+			new Object [] {"url_36","2006-10-13"},
+			new Object [] {"url_15","2004-11-5"},
+			new Object [] {"url_32","2008-2-11"},
+			new Object [] {"url_19","2009-8-3"},
+			new Object [] {"url_2","2006-8-6"},
+			new Object [] {"url_11","2009-10-13"},
+			new Object [] {"url_21","2002-9-14"},
+			new Object [] {"url_18","2000-11-2"},
+			new Object [] {"url_35","2006-5-15"},
+			new Object [] {"url_11","2006-2-18"},
+			new Object [] {"url_0","2001-4-25"},
+			new Object [] {"url_14","2009-4-8"},
+			new Object [] {"url_16","2009-4-7"}
+	};
+
 	public static DataSet<Tuple2<String, String>> getDocumentDataSet(ExecutionEnvironment env) {
-		
+
 		List<Tuple2<String, String>> data = new ArrayList<Tuple2<String, String>>(100);
-		data.add(new Tuple2<String,String>("url_0","dolor ad amet enim laoreet nostrud veniam aliquip ex nonummy diam dolore tincidunt tation exerci exerci wisi dolor nostrud "));
-		data.add(new Tuple2<String,String>("url_1","wisi minim adipiscing nibh adipiscing ut nibh Lorem Ut nonummy euismod nibh wisi sit consectetuer exerci sed aliquip aliquip dolore aliquam enim dolore veniam aliquam euismod suscipit ad adipiscing exerci aliquip consectetuer euismod aliquip ad exerci ex nibh ex erat exerci laoreet lobortis quis "));
-		data.add(new Tuple2<String,String>("url_2","diam sed convection aliquip amet commodo nonummy sed sed commodo commodo diam commodo adipiscing ad exerci magna exerci tation quis lobortis "));
-		data.add(new Tuple2<String,String>("url_3","exerci suscipit sed lobortis amet lobortis aliquip nibh nostrud ad convection commodo ad nibh sed minim amet ad ea ea "));
-		data.add(new Tuple2<String,String>("url_4","sit enim dolor quis laoreet ullamcorper veniam adipiscing ex quis commodo "));
-		data.add(new Tuple2<String,String>("url_5","elit aliquip ea nisl oscillations sit dolor ipsum tincidunt ullamcorper dolore enim adipiscing laoreet elit ea volutpat adipiscing ea nibh nostrud Ut aliquam veniam Lorem laoreet veniam aliquip "));
-		data.add(new Tuple2<String,String>("url_6","consectetuer ad sed suscipit euismod aliquip quis ullamcorper oscillations tation consectetuer tation amet suscipit nibh enim nonummy veniam commodo commodo diam euismod dolor Ut aliquip diam ex ad nonummy ad tincidunt minim exerci consectetuer veniam convection aliquam ut ut Lorem euismod sed ipsum volutpat "));
-		data.add(new Tuple2<String,String>("url_7","Ut volutpat veniam ut consectetuer diam ut aliquam dolor nostrud erat consectetuer adipiscing exerci consectetuer Ut ullamcorper suscipit aliquam sed dolor nisl "));
-		data.add(new Tuple2<String,String>("url_8","suscipit amet wisi nisl veniam lobortis sit Lorem aliquam nostrud aliquam ipsum ut laoreet suscipit Lorem laoreet editors adipiscing ullamcorper veniam erat consectetuer ut lobortis dolore elit sed tincidunt ipsum tation ullamcorper nonummy adipiscing ex ad laoreet ipsum suscipit lobortis lobortis Ut nonummy adipiscing erat volutpat aliquam "));
-		data.add(new Tuple2<String,String>("url_9","nonummy commodo tation editors ut quis sit quis lobortis ea dolore oscillations diam ad dolor lobortis nisl ad veniam ullamcorper quis magna volutpat sit ipsum consectetuer dolore exerci commodo magna erat enim ut suscipit "));
-		data.add(new Tuple2<String,String>("url_10","amet erat magna consectetuer tation tation aliquip nibh aliquam sed adipiscing ut commodo ex erat tincidunt aliquam ipsum Ut Ut sit tincidunt adipiscing suscipit minim sed erat dolor consectetuer Lorem consectetuer Lorem amet nibh diam ea ex enim suscipit wisi dolor nonummy magna enim euismod ullamcorper ut suscipit adipiscing "));
-		data.add(new Tuple2<String,String>("url_11","ex quis exerci tation diam elit nostrud nostrud ut ipsum elit amet diam laoreet amet consectetuer volutpat sed lobortis "));
-		data.add(new Tuple2<String,String>("url_12","elit suscipit sit ullamcorper ut ad erat ut dolor nostrud quis nisl enim erat dolor convection ad minim ut veniam nostrud sed editors adipiscing volutpat Ut aliquip commodo sed euismod adipiscing erat adipiscing dolore nostrud minim sed lobortis ea diam "));
-		data.add(new Tuple2<String,String>("url_13","enim ut quis commodo veniam minim erat lobortis ad diam ex dolor tincidunt exerci ut aliquip tincidunt minim ut magna sed enim wisi veniam oscillations Lorem consectetuer "));
-		data.add(new Tuple2<String,String>("url_14","nibh ipsum ullamcorper volutpat ut wisi dolor quis amet euismod quis ipsum ipsum minim tation volutpat sit exerci volutpat amet nonummy euismod veniam consectetuer sit consectetuer tincidunt nibh aliquam lobortis tation veniam ut ullamcorper wisi magna Ut volutpat consectetuer erat quis dolore ea tation "));
-		data.add(new Tuple2<String,String>("url_15","ad wisi sed enim aliquam oscillations nibh Lorem lobortis veniam nibh laoreet nonummy sed nibh Lorem adipiscing diam magna nostrud magna oscillations ut oscillations elit nostrud diam editors Lorem "));
-		data.add(new Tuple2<String,String>("url_16","nostrud volutpat veniam exerci tincidunt nostrud quis elit ipsum ea nonummy volutpat dolor elit lobortis magna nisl ut ullamcorper magna Lorem exerci nibh nisl magna editors erat aliquam aliquam ullamcorper sit aliquam sit nostrud oscillations consectetuer adipiscing suscipit convection exerci ea ullamcorper ex nisl "));
-		data.add(new Tuple2<String,String>("url_17","ad ex aliquam erat aliquam elit veniam laoreet ut amet amet nostrud ut adipiscing Ut Lorem suscipit ex magna ullamcorper aliquam ullamcorper ullamcorper amet amet commodo aliquam volutpat nonummy nonummy tincidunt amet tation tincidunt volutpat ut veniam nisl erat dolor enim nonummy nostrud adipiscing laoreet adipiscing "));
-		data.add(new Tuple2<String,String>("url_18","lobortis ipsum ex tincidunt tincidunt editors euismod consectetuer ipsum adipiscing lobortis exerci adipiscing nonummy nisl dolore nonummy erat exerci nisl ut dolore wisi volutpat lobortis magna "));
-		data.add(new Tuple2<String,String>("url_19","ipsum tation laoreet tation adipiscing wisi nibh diam Ut suscipit ad wisi "));
-		data.add(new Tuple2<String,String>("url_20","diam Lorem enim wisi ad lobortis dolor Ut ipsum amet dolore consectetuer nisl exerci nisl nonummy minim Ut erat oscillations ut Lorem nostrud dolore Ut dolore exerci ad ipsum dolore ex dolore aliquip sed aliquam ex aliquip magna amet ex dolore oscillations aliquip tation magna Ut "));
-		data.add(new Tuple2<String,String>("url_21","lobortis ut amet ex nisl ullamcorper tincidunt ut elit diam quis suscipit ad amet ipsum magna Ut ex tincidunt "));
-		data.add(new Tuple2<String,String>("url_22","amet commodo nisl ad quis lobortis ut commodo sit ut erat exerci lobortis suscipit nibh ut nostrud ut adipiscing commodo commodo quis quis nostrud nisl ipsum nostrud laoreet Lorem nostrud erat nostrud amet consectetuer laoreet oscillations wisi sit magna nibh amet "));
-		data.add(new Tuple2<String,String>("url_23","adipiscing suscipit suscipit aliquip suscipit consectetuer minim magna ea erat nibh sit suscipit sed dolor oscillations nonummy volutpat ut tincidunt "));
-		data.add(new Tuple2<String,String>("url_24","commodo sed tincidunt aliquip aliquip dolore commodo nonummy sed erat ut ex exerci dolore adipiscing tincidunt ex diam amet aliquam "));
-		data.add(new Tuple2<String,String>("url_25","consectetuer consectetuer exerci quis ea veniam aliquam laoreet minim ex "));
-		data.add(new Tuple2<String,String>("url_26","dolor exerci euismod minim magna quis erat consectetuer sed ex erat dolore quis ut oscillations ullamcorper Lorem exerci ex nibh ut exerci ullamcorper veniam nibh ut commodo ut Ut nostrud tincidunt tincidunt ad dolore Lorem ea tation enim erat nibh ut ea nonummy sed sed wisi nisl dolore "));
-		data.add(new Tuple2<String,String>("url_27","amet elit ea ea nostrud editors Ut nostrud amet laoreet adipiscing ut nisl nonummy tincidunt ea ipsum ex dolore dolore oscillations sit minim Ut wisi ut laoreet minim elit "));
-		data.add(new Tuple2<String,String>("url_28","wisi exerci volutpat Ut nostrud euismod minim Ut sit euismod ut ea magna consectetuer nisl ad minim tation nisl adipiscing Lorem aliquam quis exerci erat minim aliquip sit Lorem wisi wisi ut "));
-		data.add(new Tuple2<String,String>("url_29","amet sed laoreet amet aliquam minim enim tincidunt Lorem sit aliquip amet suscipit ut laoreet elit suscipit erat ut tincidunt suscipit ipsum sed euismod elit dolore euismod dolore ut dolor nostrud ipsum tincidunt commodo adipiscing aliquam ut wisi dolor dolor suscipit "));
-		data.add(new Tuple2<String,String>("url_30","euismod Lorem ex tincidunt amet enim minim suscipit exerci diam veniam amet nostrud ea ea "));
-		data.add(new Tuple2<String,String>("url_31","ex ipsum sit euismod euismod ullamcorper tincidunt ut wisi ea adipiscing sed diam tation ipsum dolor aliquam veniam nonummy aliquip aliquip Lorem ut minim nisl tation sit exerci ullamcorper Ut dolor euismod aliquam consectetuer ad nonummy commodo exerci "));
-		data.add(new Tuple2<String,String>("url_32","volutpat ipsum lobortis nisl veniam minim adipiscing dolor editors quis nostrud amet nostrud "));
-		data.add(new Tuple2<String,String>("url_33","commodo wisi aliquip ut aliquam sed nostrud ex diam ad nostrud enim ut amet enim ea ad sed tation nostrud suscipit ea magna magna Lorem amet lobortis ut quis nibh aliquam aliquam exerci aliquip lobortis consectetuer enim wisi ea nisl laoreet erat dolore "));
-		data.add(new Tuple2<String,String>("url_34","tincidunt adipiscing enim tation nibh Ut dolore tincidunt tation laoreet suscipit minim aliquam volutpat laoreet suscipit tincidunt nibh ut ut sit nostrud nonummy tincidunt exerci sit ad sed consectetuer minim dolor dolore laoreet nostrud nibh laoreet ea adipiscing exerci dolore ipsum "));
-		data.add(new Tuple2<String,String>("url_35","tation ut erat ut tation dolor Lorem laoreet Lorem elit adipiscing wisi aliquip nostrud elit Ut volutpat ea aliquam aliquip "));
-		data.add(new Tuple2<String,String>("url_36","lobortis enim ullamcorper adipiscing consectetuer aliquip wisi enim minim Ut minim elit elit aliquam exerci ullamcorper amet lobortis adipiscing diam laoreet consectetuer nostrud diam diam amet ut enim ullamcorper aliquip diam ut nostrud diam magna amet nonummy commodo wisi enim ullamcorper suscipit euismod dolore tincidunt magna suscipit elit "));
-		data.add(new Tuple2<String,String>("url_37","elit adipiscing nisl nisl ex aliquip nibh sed ut ad Lorem elit consectetuer ad volutpat lobortis amet veniam ipsum nibh ut consectetuer editors ad aliquam "));
-		data.add(new Tuple2<String,String>("url_38","elit quis nibh adipiscing sit consectetuer ut euismod quis tincidunt quis nisl consectetuer dolor diam suscipit quis dolore Lorem suscipit nonummy sed ex "));
-		data.add(new Tuple2<String,String>("url_39","nisl sit consectetuer elit oscillations enim ipsum enim nostrud adipiscing nostrud editors aliquam "));
-		data.add(new Tuple2<String,String>("url_40","sed wisi dolor diam commodo ullamcorper commodo nostrud ullamcorper laoreet minim dolore suscipit laoreet tation aliquip "));
-		data.add(new Tuple2<String,String>("url_41","ad consectetuer exerci nisl exerci amet enim diam lobortis Lorem ex volutpat volutpat nibh aliquam ut ullamcorper volutpat nostrud ut adipiscing ullamcorper "));
-		data.add(new Tuple2<String,String>("url_42","minim laoreet tation magna veniam ut ea sit ipsum tincidunt Ut amet ex aliquip ex euismod exerci wisi elit editors ad amet veniam ad editors "));
-		data.add(new Tuple2<String,String>("url_43","ut nisl ad ullamcorper nibh Ut editors exerci enim exerci ea laoreet veniam ea amet exerci volutpat amet ad "));
-		data.add(new Tuple2<String,String>("url_44","volutpat tincidunt enim amet sed tincidunt consectetuer ullamcorper nisl Ut adipiscing tation ad ad amet nonummy elit erat nibh Lorem erat elit laoreet consectetuer sed aliquip nostrud "));
-		data.add(new Tuple2<String,String>("url_45","sed aliquam ut ut consectetuer wisi euismod enim erat euismod quis exerci amet tation sit "));
-		data.add(new Tuple2<String,String>("url_46","lobortis oscillations tation aliquam dolore Lorem aliquip tation exerci ullamcorper aliquam aliquip lobortis ex tation dolor ut ut sed suscipit nisl ullamcorper sed editors laoreet aliquip enim dolor veniam tincidunt sed euismod tation "));
-		data.add(new Tuple2<String,String>("url_47","Lorem Lorem ut wisi ad ut tation consectetuer exerci convection tation ullamcorper sed dolore quis aliquam ipsum lobortis commodo nonummy "));
-		data.add(new Tuple2<String,String>("url_48","laoreet minim veniam nisl elit sit amet commodo ex ullamcorper suscipit aliquip laoreet convection Ut ex minim aliquam "));
-		data.add(new Tuple2<String,String>("url_49","lobortis nonummy minim amet sit veniam quis consectetuer tincidunt laoreet quis "));
-		data.add(new Tuple2<String,String>("url_50","lobortis nisl commodo dolor amet nibh editors enim magna minim elit euismod diam laoreet laoreet ad minim sed ut Ut lobortis adipiscing quis sed ut aliquam oscillations exerci tation consectetuer lobortis elit tincidunt consectetuer minim amet dolore quis aliquam Ut exerci sed aliquam quis quis ullamcorper Ut ex tincidunt "));
-		data.add(new Tuple2<String,String>("url_51","nostrud nisl ea erat ut suscipit Ut sit oscillations ullamcorper nonummy magna lobortis dolore editors tincidunt nostrud suscipit ex quis tation ut sit amet nostrud laoreet ex tincidunt "));
-		data.add(new Tuple2<String,String>("url_52","ea tation commodo elit sed ex sed quis enim nisl magna laoreet adipiscing amet sit nostrud consectetuer nibh tincidunt veniam ex veniam euismod exerci sed dolore suscipit nisl tincidunt euismod quis Ut enim euismod dolor diam exerci magna exerci ut exerci nisl "));
-		data.add(new Tuple2<String,String>("url_53","volutpat amet Ut lobortis dolor tation minim nonummy lobortis convection nostrud "));
-		data.add(new Tuple2<String,String>("url_54","ullamcorper commodo Ut amet sit nostrud aliquam ad amet wisi enim nostrud ipsum nisl veniam erat aliquam ex aliquam dolor dolor ut consectetuer euismod exerci elit exerci Ut ea minim enim consectetuer ad consectetuer nonummy convection adipiscing ad ullamcorper lobortis nonummy laoreet nonummy aliquam ullamcorper ad nostrud amet "));
-		data.add(new Tuple2<String,String>("url_55","wisi magna editors amet aliquam diam amet aliquip nisl consectetuer laoreet nonummy suscipit euismod diam enim tation elit ut lobortis quis euismod suscipit nostrud ea ea commodo lobortis dolore Ut nisl nostrud dolor laoreet euismod ea dolore aliquam ut Lorem exerci ex sit "));
-		data.add(new Tuple2<String,String>("url_56","ex dolor veniam wisi laoreet ut exerci diam ad ex ut ut laoreet ut nisl ullamcorper nisl "));
-		data.add(new Tuple2<String,String>("url_57","diam adipiscing Ut ut Lorem amet erat elit erat magna adipiscing euismod elit ullamcorper nostrud aliquam dolor ullamcorper sit tation tation "));
-		data.add(new Tuple2<String,String>("url_58","laoreet convection veniam lobortis dolore ut nonummy commodo erat lobortis veniam nostrud dolore minim commodo ut consectetuer magna erat ea dolore Lorem suscipit ex ipsum exerci sed enim ea tation suscipit enim adipiscing "));
-		data.add(new Tuple2<String,String>("url_59","amet ut ut Ut ad dolor quis ad magna exerci suscipit magna nibh commodo euismod amet euismod wisi diam suscipit dolore Lorem dolor ex amet exerci aliquip ut ut lobortis quis elit minim sed Lorem "));
-		data.add(new Tuple2<String,String>("url_60","ut ut amet ullamcorper amet euismod dolor amet elit exerci adipiscing sed suscipit sed exerci wisi diam veniam wisi suscipit ut quis nibh ullamcorper ex quis magna dolore volutpat editors minim ut sit aliquip oscillations nisl ipsum "));
-		data.add(new Tuple2<String,String>("url_61","nibh nostrud tincidunt lobortis adipiscing adipiscing ullamcorper ullamcorper ipsum nisl ullamcorper aliquip laoreet commodo ut tation wisi diam commodo aliquip commodo suscipit tincidunt volutpat elit enim laoreet ut nostrud ad nonummy ipsum "));
-		data.add(new Tuple2<String,String>("url_62","Ut ut minim enim amet euismod erat elit commodo consectetuer Ut quis dolor ex diam quis wisi tation tincidunt laoreet volutpat "));
-		data.add(new Tuple2<String,String>("url_63","ut erat volutpat euismod amet ea nonummy lobortis ut Ut ea veniam sed veniam nostrud "));
-		data.add(new Tuple2<String,String>("url_64","tation dolor suscipit minim nisl wisi consectetuer aliquip tation Ut commodo ut dolore consectetuer elit wisi nisl ipsum "));
-		data.add(new Tuple2<String,String>("url_65","ullamcorper nisl Lorem magna tation veniam aliquam diam amet euismod "));
-		data.add(new Tuple2<String,String>("url_66","euismod aliquam tincidunt Ut volutpat ea lobortis sit ut volutpat ut lobortis ut lobortis ut nisl amet dolor sed ipsum enim ullamcorper diam euismod nostrud wisi erat quis diam nibh Ut dolore sed amet tation enim diam "));
-		data.add(new Tuple2<String,String>("url_67","amet minim minim amet laoreet Lorem aliquam veniam elit volutpat magna adipiscing enim enim euismod laoreet sed ex sed aliquam ad ea ut adipiscing suscipit ex minim dolore minim ea laoreet nisl "));
-		data.add(new Tuple2<String,String>("url_68","aliquam ea volutpat ut wisi tation tation nibh nisl erat laoreet ea volutpat dolor dolor aliquam exerci quis ullamcorper aliquam ut quis suscipit "));
-		data.add(new Tuple2<String,String>("url_69","quis exerci ut aliquip wisi dolore magna nibh consectetuer magna tation ullamcorper lobortis sed amet adipiscing minim suscipit nibh nibh nostrud euismod enim "));
-		data.add(new Tuple2<String,String>("url_70","tation enim consectetuer adipiscing wisi laoreet diam aliquip nostrud elit nostrud aliquip ea minim amet diam dolore "));
-		data.add(new Tuple2<String,String>("url_71","consectetuer tincidunt nibh amet tation nonummy sit tation diam sed diam tation "));
-		data.add(new Tuple2<String,String>("url_72","Lorem ut nostrud nonummy minim quis euismod lobortis nostrud nonummy adipiscing tincidunt consectetuer ut nibh ad suscipit dolor ut elit dolore amet ut quis tation ullamcorper nonummy laoreet ullamcorper aliquam dolore convection dolor tincidunt ut ullamcorper ex dolor suscipit erat oscillations ad "));
-		data.add(new Tuple2<String,String>("url_73","elit Ut commodo ut ullamcorper ullamcorper ut euismod commodo diam aliquip suscipit consectetuer exerci tation nostrud ut wisi exerci sed ut elit sed volutpat Lorem nibh laoreet consectetuer ex Lorem elit aliquam commodo lobortis ad "));
-		data.add(new Tuple2<String,String>("url_74","quis magna laoreet commodo aliquam nisl ullamcorper veniam tation wisi consectetuer commodo consectetuer ad dolore aliquam dolor elit amet sit amet nibh commodo erat veniam aliquip dolore ad magna ad ipsum Ut exerci ea volutpat nisl amet nostrud sit "));
-		data.add(new Tuple2<String,String>("url_75","tincidunt suscipit sit aliquip aliquam adipiscing dolore exerci Ut suscipit ut sit laoreet suscipit wisi sit enim nonummy consectetuer dolore editors "));
-		data.add(new Tuple2<String,String>("url_76","veniam ullamcorper tation sit suscipit dolor suscipit veniam sit Lorem quis sed nostrud ad tincidunt elit adipiscing "));
-		data.add(new Tuple2<String,String>("url_77","volutpat sit amet veniam quis ipsum nibh elit enim commodo magna veniam magna convection "));
-		data.add(new Tuple2<String,String>("url_78","tation dolore minim elit nisl volutpat tation laoreet enim nostrud exerci dolore tincidunt aliquip Lorem ipsum nostrud quis adipiscing ullamcorper erat lobortis tation commodo Ut ipsum commodo magna ad ipsum ut enim "));
-		data.add(new Tuple2<String,String>("url_79","lobortis amet elit Lorem amet nonummy commodo tation ex ea amet Lorem ea nonummy commodo veniam volutpat nibh wisi ad ipsum euismod ea convection nostrud nisl erat veniam Ut aliquip ad aliquip editors wisi magna tation nostrud nonummy adipiscing ullamcorper aliquip "));
-		data.add(new Tuple2<String,String>("url_80","tincidunt nostrud nostrud magna ea euismod ea consectetuer nisl exerci ea dolor nisl commodo ex erat ipsum exerci suscipit ad nisl ea nonummy suscipit adipiscing laoreet sit euismod nibh adipiscing sed minim commodo amet "));
-		data.add(new Tuple2<String,String>("url_81","nostrud erat ut sed editors erat amet magna lobortis diam laoreet dolor amet nibh ut ipsum ipsum amet ut sed ut exerci elit suscipit wisi magna ut veniam nisl commodo enim adipiscing laoreet ad Lorem oscillations "));
-		data.add(new Tuple2<String,String>("url_82","quis commodo nibh nibh volutpat suscipit dolore magna tincidunt nibh ut ad ullamcorper ullamcorper quis enim ad ut tation minim laoreet veniam dolor sed tincidunt exerci exerci nostrud ullamcorper amet ut ut ullamcorper "));
-		data.add(new Tuple2<String,String>("url_83","sit suscipit volutpat elit tation elit sed sed dolor ex ex ipsum euismod laoreet magna lobortis ad "));
-		data.add(new Tuple2<String,String>("url_84","lobortis ipsum euismod enim ea tation veniam tation oscillations aliquip consectetuer euismod ut sed lobortis tation oscillations commodo euismod laoreet suscipit amet elit ullamcorper volutpat aliquam ea enim ullamcorper consectetuer laoreet tation quis ut commodo erat euismod dolor laoreet ullamcorper laoreet "));
-		data.add(new Tuple2<String,String>("url_85","adipiscing sit quis commodo consectetuer quis enim euismod exerci nonummy ea nostrud Ut veniam sit aliquip nisl enim "));
-		data.add(new Tuple2<String,String>("url_86","nostrud dolore veniam veniam wisi aliquip adipiscing diam sed quis ullamcorper "));
-		data.add(new Tuple2<String,String>("url_87","quis Lorem suscipit Ut nibh diam euismod consectetuer lobortis ipsum sed suscipit consectetuer euismod laoreet ut wisi nisl elit quis commodo adipiscing adipiscing suscipit aliquam nisl quis magna ipsum enim ad quis ea magna Lorem nibh ea "));
-		data.add(new Tuple2<String,String>("url_88","euismod commodo sed tincidunt Ut veniam consectetuer quis erat ex ea erat laoreet commodo nibh minim "));
-		data.add(new Tuple2<String,String>("url_89","tation diam editors Ut enim nibh Lorem volutpat quis diam suscipit exerci wisi ad "));
-		data.add(new Tuple2<String,String>("url_90","volutpat editors ea nibh wisi ad amet volutpat nisl ullamcorper nibh volutpat minim ex ut sit veniam Lorem consectetuer quis ad sit suscipit volutpat wisi diam sed tincidunt ipsum minim convection ea diam oscillations quis lobortis "));
-		data.add(new Tuple2<String,String>("url_91","enim minim nonummy ea minim euismod adipiscing editors volutpat magna sit magna ut ipsum ut "));
-		data.add(new Tuple2<String,String>("url_92","nisl Ut commodo amet euismod lobortis ea ea wisi commodo Lorem sit ipsum volutpat nonummy exerci erat elit exerci magna ad erat enim laoreet quis nostrud wisi ut veniam amet ullamcorper lobortis ad suscipit volutpat veniam nostrud nibh quis ipsum dolore consectetuer veniam ipsum aliquip dolore sed laoreet ipsum "));
-		data.add(new Tuple2<String,String>("url_93","nonummy aliquam ad lobortis Lorem erat ad tation Lorem exerci ex "));
-		data.add(new Tuple2<String,String>("url_94","nonummy dolore commodo exerci ex quis ut suscipit elit laoreet sit tation magna veniam ea sit nonummy veniam Lorem quis nibh aliquip exerci amet ullamcorper adipiscing erat nisl editors diam commodo ad euismod adipiscing ea suscipit exerci aliquip volutpat tation enim volutpat sit "));
-		data.add(new Tuple2<String,String>("url_95","sit suscipit oscillations ipsum nibh dolor ea dolore ea elit ipsum minim editors magna consectetuer ullamcorper commodo nonummy sit nostrud aliquip sit erat ullamcorper ullamcorper nibh veniam erat quis dolore nonummy "));
-		data.add(new Tuple2<String,String>("url_96","nostrud quis ut volutpat magna ad quis adipiscing Lorem commodo exerci laoreet magna adipiscing erat quis wisi ea ea laoreet enim convection ad dolor nisl amet nibh aliquam adipiscing tincidunt minim diam Lorem commodo adipiscing volutpat "));
-		data.add(new Tuple2<String,String>("url_97","laoreet laoreet suscipit nostrud dolore adipiscing volutpat Ut sed nisl diam ullamcorper ex ut ut dolor amet nostrud euismod dolore veniam veniam enim tation veniam ea minim minim volutpat tincidunt "));
-		data.add(new Tuple2<String,String>("url_98","quis lobortis amet wisi nostrud ipsum aliquam convection tincidunt dolore ullamcorper nibh lobortis volutpat ea nostrud oscillations minim nonummy enim ad lobortis exerci ipsum ullamcorper nibh nonummy diam amet enim veniam ut nostrud "));
-		data.add(new Tuple2<String,String>("url_99","aliquam wisi suscipit commodo diam amet amet magna nisl enim nostrud tation nisl nostrud nibh ut "));
-		
-		return env.fromCollection(data);	
+		for (Object [] document : DOCUMENTS) {
+			data.add(new Tuple2<String, String>((String) document[0], (String) document[1]));
+		}
+
+		return env.fromCollection(data);
 	}
-	
+
 	public static DataSet<Tuple3<Integer, String, Integer>> getRankDataSet(ExecutionEnvironment env) {
-	
+
 		List<Tuple3<Integer, String, Integer>> data = new ArrayList<Tuple3<Integer, String, Integer>>(100);
-		data.add(new Tuple3<Integer, String, Integer>(30,"url_0",43));
-		data.add(new Tuple3<Integer, String, Integer>(82,"url_1",39));
-		data.add(new Tuple3<Integer, String, Integer>(56,"url_2",31));
-		data.add(new Tuple3<Integer, String, Integer>(96,"url_3",36));
-		data.add(new Tuple3<Integer, String, Integer>(31,"url_4",36));
-		data.add(new Tuple3<Integer, String, Integer>(29,"url_5",6));
-		data.add(new Tuple3<Integer, String, Integer>(33,"url_6",48));
-		data.add(new Tuple3<Integer, String, Integer>(66,"url_7",40));
-		data.add(new Tuple3<Integer, String, Integer>(28,"url_8",51));
-		data.add(new Tuple3<Integer, String, Integer>(9,"url_9",4));
-		data.add(new Tuple3<Integer, String, Integer>(49,"url_10",24));
-		data.add(new Tuple3<Integer, String, Integer>(26,"url_11",12));
-		data.add(new Tuple3<Integer, String, Integer>(39,"url_12",46));
-		data.add(new Tuple3<Integer, String, Integer>(84,"url_13",53));
-		data.add(new Tuple3<Integer, String, Integer>(29,"url_14",50));
-		data.add(new Tuple3<Integer, String, Integer>(21,"url_15",12));
-		data.add(new Tuple3<Integer, String, Integer>(69,"url_16",34));
-		data.add(new Tuple3<Integer, String, Integer>(11,"url_17",38));
-		data.add(new Tuple3<Integer, String, Integer>(96,"url_18",13));
-		data.add(new Tuple3<Integer, String, Integer>(56,"url_19",48));
-		data.add(new Tuple3<Integer, String, Integer>(18,"url_20",36));
-		data.add(new Tuple3<Integer, String, Integer>(31,"url_21",21));
-		data.add(new Tuple3<Integer, String, Integer>(29,"url_22",11));
-		data.add(new Tuple3<Integer, String, Integer>(71,"url_23",30));
-		data.add(new Tuple3<Integer, String, Integer>(85,"url_24",48));
-		data.add(new Tuple3<Integer, String, Integer>(19,"url_25",45));
-		data.add(new Tuple3<Integer, String, Integer>(69,"url_26",9));
-		data.add(new Tuple3<Integer, String, Integer>(20,"url_27",51));
-		data.add(new Tuple3<Integer, String, Integer>(33,"url_28",46));
-		data.add(new Tuple3<Integer, String, Integer>(75,"url_29",38));
-		data.add(new Tuple3<Integer, String, Integer>(96,"url_30",51));
-		data.add(new Tuple3<Integer, String, Integer>(73,"url_31",40));
-		data.add(new Tuple3<Integer, String, Integer>(67,"url_32",16));
-		data.add(new Tuple3<Integer, String, Integer>(24,"url_33",24));
-		data.add(new Tuple3<Integer, String, Integer>(27,"url_34",35));
-		data.add(new Tuple3<Integer, String, Integer>(33,"url_35",35));
-		data.add(new Tuple3<Integer, String, Integer>(7,"url_36",22));
-		data.add(new Tuple3<Integer, String, Integer>(83,"url_37",41));
-		data.add(new Tuple3<Integer, String, Integer>(23,"url_38",49));
-		data.add(new Tuple3<Integer, String, Integer>(41,"url_39",33));
-		data.add(new Tuple3<Integer, String, Integer>(66,"url_40",38));
-		data.add(new Tuple3<Integer, String, Integer>(4,"url_41",52));
-		data.add(new Tuple3<Integer, String, Integer>(34,"url_42",4));
-		data.add(new Tuple3<Integer, String, Integer>(28,"url_43",12));
-		data.add(new Tuple3<Integer, String, Integer>(14,"url_44",14));
-		data.add(new Tuple3<Integer, String, Integer>(41,"url_45",11));
-		data.add(new Tuple3<Integer, String, Integer>(48,"url_46",37));
-		data.add(new Tuple3<Integer, String, Integer>(75,"url_47",41));
-		data.add(new Tuple3<Integer, String, Integer>(78,"url_48",3));
-		data.add(new Tuple3<Integer, String, Integer>(63,"url_49",28));
-	
+		for (Object [] rank : RANKS) {
+			data.add(new Tuple3<Integer, String, Integer>((Integer) rank[0], (String) rank[1], (Integer) rank[2]));
+		}
 		return env.fromCollection(data);
 	}
-	
+
 	public static DataSet<Tuple2<String, String>> getVisitDataSet(ExecutionEnvironment env) {
-		
+
 		List<Tuple2<String, String>> data = new ArrayList<Tuple2<String, String>>(100);
-		data.add(new Tuple2<String, String>("url_2","2003-12-17"));
-		data.add(new Tuple2<String, String>("url_9","2008-11-11"));
-		data.add(new Tuple2<String, String>("url_14","2003-11-5"));
-		data.add(new Tuple2<String, String>("url_46","2009-2-16"));
-		data.add(new Tuple2<String, String>("url_14","2004-11-9"));
-		data.add(new Tuple2<String, String>("url_36","2001-3-9"));
-		data.add(new Tuple2<String, String>("url_35","2006-8-13"));
-		data.add(new Tuple2<String, String>("url_22","2008-1-18"));
-		data.add(new Tuple2<String, String>("url_36","2002-3-9"));
-		data.add(new Tuple2<String, String>("url_13","2007-7-17"));
-		data.add(new Tuple2<String, String>("url_23","2009-6-16"));
-		data.add(new Tuple2<String, String>("url_16","2000-7-15"));
-		data.add(new Tuple2<String, String>("url_41","2002-5-10"));
-		data.add(new Tuple2<String, String>("url_6","2004-11-9"));
-		data.add(new Tuple2<String, String>("url_5","2003-6-7"));
-		data.add(new Tuple2<String, String>("url_22","2002-11-5"));
-		data.add(new Tuple2<String, String>("url_11","2007-7-21"));
-		data.add(new Tuple2<String, String>("url_38","2009-12-2"));
-		data.add(new Tuple2<String, String>("url_6","2004-11-2"));
-		data.add(new Tuple2<String, String>("url_46","2000-6-4"));
-		data.add(new Tuple2<String, String>("url_34","2003-9-2"));
-		data.add(new Tuple2<String, String>("url_31","2008-2-24"));
-		data.add(new Tuple2<String, String>("url_0","2003-2-2"));
-		data.add(new Tuple2<String, String>("url_47","2003-7-8"));
-		data.add(new Tuple2<String, String>("url_49","2009-9-13"));
-		data.add(new Tuple2<String, String>("url_11","2003-4-2"));
-		data.add(new Tuple2<String, String>("url_20","2000-6-18"));
-		data.add(new Tuple2<String, String>("url_38","2000-2-22"));
-		data.add(new Tuple2<String, String>("url_44","2009-2-17"));
-		data.add(new Tuple2<String, String>("url_26","2000-6-21"));
-		data.add(new Tuple2<String, String>("url_13","2000-11-25"));
-		data.add(new Tuple2<String, String>("url_47","2005-4-19"));
-		data.add(new Tuple2<String, String>("url_46","2008-1-7"));
-		data.add(new Tuple2<String, String>("url_33","2004-12-24"));
-		data.add(new Tuple2<String, String>("url_32","2009-2-8"));
-		data.add(new Tuple2<String, String>("url_26","2000-9-21"));
-		data.add(new Tuple2<String, String>("url_9","2002-8-18"));
-		data.add(new Tuple2<String, String>("url_38","2002-11-27"));
-		data.add(new Tuple2<String, String>("url_37","2008-2-26"));
-		data.add(new Tuple2<String, String>("url_1","2007-3-22"));
-		data.add(new Tuple2<String, String>("url_37","2002-3-20"));
-		data.add(new Tuple2<String, String>("url_27","2008-11-12"));
-		data.add(new Tuple2<String, String>("url_30","2000-12-16"));
-		data.add(new Tuple2<String, String>("url_48","2000-12-17"));
-		data.add(new Tuple2<String, String>("url_46","2008-4-16"));
-		data.add(new Tuple2<String, String>("url_29","2006-3-9"));
-		data.add(new Tuple2<String, String>("url_0","2007-7-26"));
-		data.add(new Tuple2<String, String>("url_46","2009-12-15"));
-		data.add(new Tuple2<String, String>("url_34","2002-2-13"));
-		data.add(new Tuple2<String, String>("url_24","2009-3-1"));
-		data.add(new Tuple2<String, String>("url_43","2007-11-4"));
-		data.add(new Tuple2<String, String>("url_3","2004-2-16"));
-		data.add(new Tuple2<String, String>("url_26","2000-10-26"));
-		data.add(new Tuple2<String, String>("url_42","2004-7-14"));
-		data.add(new Tuple2<String, String>("url_13","2004-9-10"));
-		data.add(new Tuple2<String, String>("url_21","2000-2-21"));
-		data.add(new Tuple2<String, String>("url_9","2006-6-5"));
-		data.add(new Tuple2<String, String>("url_46","2001-12-17"));
-		data.add(new Tuple2<String, String>("url_24","2006-12-8"));
-		data.add(new Tuple2<String, String>("url_25","2006-9-2"));
-		data.add(new Tuple2<String, String>("url_37","2002-6-26"));
-		data.add(new Tuple2<String, String>("url_18","2006-6-2"));
-		data.add(new Tuple2<String, String>("url_46","2003-5-24"));
-		data.add(new Tuple2<String, String>("url_32","2000-10-17"));
-		data.add(new Tuple2<String, String>("url_45","2002-1-12"));
-		data.add(new Tuple2<String, String>("url_12","2005-12-13"));
-		data.add(new Tuple2<String, String>("url_49","2009-3-9"));
-		data.add(new Tuple2<String, String>("url_31","2001-9-19"));
-		data.add(new Tuple2<String, String>("url_22","2002-7-9"));
-		data.add(new Tuple2<String, String>("url_27","2005-2-3"));
-		data.add(new Tuple2<String, String>("url_43","2008-7-15"));
-		data.add(new Tuple2<String, String>("url_20","2000-3-23"));
-		data.add(new Tuple2<String, String>("url_25","2002-5-8"));
-		data.add(new Tuple2<String, String>("url_41","2004-4-27"));
-		data.add(new Tuple2<String, String>("url_17","2008-7-17"));
-		data.add(new Tuple2<String, String>("url_26","2009-12-16"));
-		data.add(new Tuple2<String, String>("url_34","2006-2-10"));
-		data.add(new Tuple2<String, String>("url_8","2009-4-14"));
-		data.add(new Tuple2<String, String>("url_16","2000-2-24"));
-		data.add(new Tuple2<String, String>("url_2","2009-2-10"));
-		data.add(new Tuple2<String, String>("url_35","2003-2-24"));
-		data.add(new Tuple2<String, String>("url_34","2008-3-16"));
-		data.add(new Tuple2<String, String>("url_27","2005-1-5"));
-		data.add(new Tuple2<String, String>("url_8","2008-12-10"));
-		data.add(new Tuple2<String, String>("url_38","2009-2-11"));
-		data.add(new Tuple2<String, String>("url_38","2006-11-3"));
-		data.add(new Tuple2<String, String>("url_47","2003-2-13"));
-		data.add(new Tuple2<String, String>("url_8","2008-11-17"));
-		data.add(new Tuple2<String, String>("url_26","2009-5-11"));
-		data.add(new Tuple2<String, String>("url_12","2007-11-26"));
-		data.add(new Tuple2<String, String>("url_10","2003-1-13"));
-		data.add(new Tuple2<String, String>("url_8","2005-9-23"));
-		data.add(new Tuple2<String, String>("url_42","2001-4-5"));
-		data.add(new Tuple2<String, String>("url_30","2009-12-10"));
-		data.add(new Tuple2<String, String>("url_2","2003-1-3"));
-		data.add(new Tuple2<String, String>("url_2","2009-2-19"));
-		data.add(new Tuple2<String, String>("url_7","2000-6-25"));
-		data.add(new Tuple2<String, String>("url_15","2004-9-26"));
-		data.add(new Tuple2<String, String>("url_25","2009-10-5"));
-		data.add(new Tuple2<String, String>("url_23","2009-8-9"));
-		data.add(new Tuple2<String, String>("url_27","2004-4-3"));
-		data.add(new Tuple2<String, String>("url_37","2008-6-9"));
-		data.add(new Tuple2<String, String>("url_9","2002-5-25"));
-		data.add(new Tuple2<String, String>("url_43","2009-5-18"));
-		data.add(new Tuple2<String, String>("url_21","2008-4-19"));
-		data.add(new Tuple2<String, String>("url_12","2001-12-25"));
-		data.add(new Tuple2<String, String>("url_16","2006-9-25"));
-		data.add(new Tuple2<String, String>("url_27","2002-1-2"));
-		data.add(new Tuple2<String, String>("url_2","2009-1-21"));
-		data.add(new Tuple2<String, String>("url_31","2009-3-20"));
-		data.add(new Tuple2<String, String>("url_42","2002-3-1"));
-		data.add(new Tuple2<String, String>("url_31","2001-11-26"));
-		data.add(new Tuple2<String, String>("url_20","2003-5-15"));
-		data.add(new Tuple2<String, String>("url_32","2004-1-22"));
-		data.add(new Tuple2<String, String>("url_28","2008-9-16"));
-		data.add(new Tuple2<String, String>("url_27","2006-7-3"));
-		data.add(new Tuple2<String, String>("url_11","2008-12-26"));
-		data.add(new Tuple2<String, String>("url_15","2004-8-16"));
-		data.add(new Tuple2<String, String>("url_34","2002-10-5"));
-		data.add(new Tuple2<String, String>("url_44","2000-2-15"));
-		data.add(new Tuple2<String, String>("url_9","2000-10-23"));
-		data.add(new Tuple2<String, String>("url_45","2005-4-24"));
-		data.add(new Tuple2<String, String>("url_0","2006-8-7"));
-		data.add(new Tuple2<String, String>("url_48","2003-8-7"));
-		data.add(new Tuple2<String, String>("url_8","2007-12-13"));
-		data.add(new Tuple2<String, String>("url_42","2003-8-2"));
-		data.add(new Tuple2<String, String>("url_25","2008-3-5"));
-		data.add(new Tuple2<String, String>("url_3","2007-3-9"));
-		data.add(new Tuple2<String, String>("url_49","2003-10-7"));
-		data.add(new Tuple2<String, String>("url_18","2007-12-6"));
-		data.add(new Tuple2<String, String>("url_3","2006-7-5"));
-		data.add(new Tuple2<String, String>("url_27","2000-9-14"));
-		data.add(new Tuple2<String, String>("url_42","2002-10-20"));
-		data.add(new Tuple2<String, String>("url_44","2007-1-13"));
-		data.add(new Tuple2<String, String>("url_6","2003-1-21"));
-		data.add(new Tuple2<String, String>("url_40","2009-10-20"));
-		data.add(new Tuple2<String, String>("url_28","2009-6-17"));
-		data.add(new Tuple2<String, String>("url_22","2000-2-17"));
-		data.add(new Tuple2<String, String>("url_3","2005-1-15"));
-		data.add(new Tuple2<String, String>("url_9","2008-12-9"));
-		data.add(new Tuple2<String, String>("url_9","2005-2-19"));
-		data.add(new Tuple2<String, String>("url_28","2000-4-22"));
-		data.add(new Tuple2<String, String>("url_44","2001-9-9"));
-		data.add(new Tuple2<String, String>("url_43","2008-6-21"));
-		data.add(new Tuple2<String, String>("url_39","2008-5-9"));
-		data.add(new Tuple2<String, String>("url_15","2006-9-15"));
-		data.add(new Tuple2<String, String>("url_23","2001-12-18"));
-		data.add(new Tuple2<String, String>("url_14","2002-5-23"));
-		data.add(new Tuple2<String, String>("url_11","2007-7-11"));
-		data.add(new Tuple2<String, String>("url_34","2000-12-8"));
-		data.add(new Tuple2<String, String>("url_47","2005-7-3"));
-		data.add(new Tuple2<String, String>("url_38","2004-3-26"));
-		data.add(new Tuple2<String, String>("url_19","2003-9-14"));
-		data.add(new Tuple2<String, String>("url_24","2007-7-16"));
-		data.add(new Tuple2<String, String>("url_40","2008-8-21"));
-		data.add(new Tuple2<String, String>("url_17","2007-12-4"));
-		data.add(new Tuple2<String, String>("url_25","2006-6-24"));
-		data.add(new Tuple2<String, String>("url_2","2000-10-8"));
-		data.add(new Tuple2<String, String>("url_12","2008-6-10"));
-		data.add(new Tuple2<String, String>("url_11","2004-11-24"));
-		data.add(new Tuple2<String, String>("url_13","2005-11-3"));
-		data.add(new Tuple2<String, String>("url_43","2005-1-2"));
-		data.add(new Tuple2<String, String>("url_14","2008-6-12"));
-		data.add(new Tuple2<String, String>("url_43","2001-8-27"));
-		data.add(new Tuple2<String, String>("url_45","2000-3-3"));
-		data.add(new Tuple2<String, String>("url_0","2006-9-27"));
-		data.add(new Tuple2<String, String>("url_22","2007-12-18"));
-		data.add(new Tuple2<String, String>("url_25","2006-4-4"));
-		data.add(new Tuple2<String, String>("url_32","2001-6-25"));
-		data.add(new Tuple2<String, String>("url_6","2007-6-9"));
-		data.add(new Tuple2<String, String>("url_8","2009-10-3"));
-		data.add(new Tuple2<String, String>("url_15","2003-2-23"));
-		data.add(new Tuple2<String, String>("url_37","2000-5-6"));
-		data.add(new Tuple2<String, String>("url_27","2004-3-21"));
-		data.add(new Tuple2<String, String>("url_17","2005-6-20"));
-		data.add(new Tuple2<String, String>("url_2","2004-2-27"));
-		data.add(new Tuple2<String, String>("url_36","2005-3-16"));
-		data.add(new Tuple2<String, String>("url_1","2009-12-3"));
-		data.add(new Tuple2<String, String>("url_9","2004-4-27"));
-		data.add(new Tuple2<String, String>("url_18","2009-5-26"));
-		data.add(new Tuple2<String, String>("url_31","2000-9-21"));
-		data.add(new Tuple2<String, String>("url_12","2008-9-25"));
-		data.add(new Tuple2<String, String>("url_2","2004-2-16"));
-		data.add(new Tuple2<String, String>("url_28","2008-11-12"));
-		data.add(new Tuple2<String, String>("url_28","2001-6-26"));
-		data.add(new Tuple2<String, String>("url_12","2006-3-15"));
-		data.add(new Tuple2<String, String>("url_0","2009-3-1"));
-		data.add(new Tuple2<String, String>("url_36","2006-10-13"));
-		data.add(new Tuple2<String, String>("url_15","2004-11-5"));
-		data.add(new Tuple2<String, String>("url_32","2008-2-11"));
-		data.add(new Tuple2<String, String>("url_19","2009-8-3"));
-		data.add(new Tuple2<String, String>("url_2","2006-8-6"));
-		data.add(new Tuple2<String, String>("url_11","2009-10-13"));
-		data.add(new Tuple2<String, String>("url_21","2002-9-14"));
-		data.add(new Tuple2<String, String>("url_18","2000-11-2"));
-		data.add(new Tuple2<String, String>("url_35","2006-5-15"));
-		data.add(new Tuple2<String, String>("url_11","2006-2-18"));
-		data.add(new Tuple2<String, String>("url_0","2001-4-25"));
-		data.add(new Tuple2<String, String>("url_14","2009-4-8"));
-		data.add(new Tuple2<String, String>("url_16","2009-4-7"));
-		
+
+		for (Object [] visit : VISITS) {
+			data.add(new Tuple2<String, String>((String) visit[0], (String) visit[1]));
+		}
 		return env.fromCollection(data);
-		
+
 	}
-	
-	
-}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a8dd9587/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
index e1ad641..f84b54c 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
@@ -1,150 +1,198 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.relational;
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-///**
-// * Implements the following relational OLAP query as PACT program:
-// *
-// * <code><pre>
-// * SELECT r.pageURL, r.pageRank, r.avgDuration
-// * FROM Documents d JOIN Rankings r
-// * 	ON d.url = r.url
-// * WHERE CONTAINS(d.text, [keywords])
-// * 	AND r.rank > [rank]
-// * 	AND NOT EXISTS (
-// * 		SELECT * FROM Visits v
-// * 		WHERE v.destUrl = d.url
-// * 			AND v.visitDate < [date]);
-// *  * </pre></code>
-// *
-// * Table Schemas: <code><pre>
-// * CREATE TABLE Documents (
-// * 					url VARCHAR(100) PRIMARY KEY,
-// * 					contents TEXT );
-// *
-// * CREATE TABLE Rankings (
-// * 					pageRank INT,
-// * 					pageURL VARCHAR(100) PRIMARY KEY,
-// * 					avgDuration INT );
-// *
-// * CREATE TABLE Visits (
-// * 					sourceIP VARCHAR(16),
-// * 					destURL VARCHAR(100),
-// * 					visitDate DATE,
-// * 					adRevenue FLOAT,
-// * 					userAgent VARCHAR(64),
-// * 					countryCode VARCHAR(3),
-// * 					languageCode VARCHAR(6),
-// * 					searchWord VARCHAR(32),
-// * 					duration INT );
-// * </pre></code>
-// *
-// */
-//class WebLogAnalysis extends Program with ProgramDescription with Serializable {
-//
-//  override def getDescription() = {
-//    "Parameters: [numSubStasks], [docs], [rankings], [visits], [output]"
-//  }
-//
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4))
-//  }
-//
-//  // document tuple
-//  case class Doc(url: String, text: String)
-//
-//  // rank tuple
-//  case class Rank(rank: Int, url: String, avgDuration: Int)
-//
-//  // visit tuple
-//  case class Visit(url: String, date: String)
-//
-//  // format rank for output
-//  def formatRank = (r: Rank) => "%d|%s|%d".format(r.rank, r.url, r.avgDuration)
-//
-//  def getScalaPlan(numSubTasks: Int, docsInput: String, rankingsInput: String, visitsInput: String, ranksOutput: String) = {
-//
-//    // read documents data
-//    val docs = DataSource(docsInput, CsvInputFormat[Doc]("\n", '|'))
-//    // read ranks data
-//    val ranks = DataSource(rankingsInput, CsvInputFormat[Rank]("\n", '|'))
-//    // read visits data and project to visits tuple afterwards
-//    val visits = DataSource(visitsInput, CsvInputFormat[(String, String, String)]("\n", '|')) map (x => Visit(x._2, x._3))
-//
-//    // filter on documents that contain certain key words and project to URL
-//    val filteredDocs = docs filter {d => d.text.contains(" editors ") && d.text.contains(" oscillations ") && d.text.contains(" convection ")} map { d => d.url }
-//    filteredDocs.filterFactor(.15f)
-//    filteredDocs.observes(d => (d.text))
-//    filteredDocs.preserves({d => d.url}, {url => url} )
-//
-//    // filter on ranks that have a certain minimum rank
-//    val filteredRanks = ranks filter {r => r.rank > 50}
-//    filteredRanks.filterFactor(.25f)
-//    filteredRanks.observes(r => (r.rank))
-//    filteredRanks.preserves({r => r}, {r => r} )
-//
-//    // filter on visits of the year 2010 and project to URL
-//    val filteredVisits = visits filter {v => v.date.substring(0,4).equals("2010")} map { v => v.url }
-//    filteredVisits.filterFactor(.2f)
-//    filteredVisits.observes(v => (v.date))
-//    filteredVisits.preserves( {v => v.url}, {url => url} )
-//
-//    // filter for ranks on documents that contain certain key words
-//    val ranksFilteredByDocs = filteredDocs join filteredRanks where {url => url} isEqualTo {r => r.url} map ((d,r) => r)
-//    ranksFilteredByDocs.left.neglects( {d => d} )
-//    ranksFilteredByDocs.right.preserves( {r => r}, {r => r} )
-//
-//    // filter for ranks on documents that have not been visited in 2010
-//    val ranksFilteredByDocsAndVisits = ranksFilteredByDocs cogroup filteredVisits where {r => r.url} isEqualTo {url => url} map ( (rs, vs) => if (vs.hasNext) Nil else rs.toList ) flatMap {rs => rs.iterator }
-//    ranksFilteredByDocs.left.preserves( {r => r}, {r => r} )
-//    ranksFilteredByDocs.right.neglects( {v => v} )
-//
-//    // emit the resulting ranks
-//    val output = ranksFilteredByDocsAndVisits.write(ranksOutput, DelimitedOutputFormat(formatRank))
-//
-//    val plan = new ScalaPlan(Seq(output), "WebLog Analysis")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//}
-//
-//
-///**
-// * Entry point to make the example standalone runnable with the local executor
-// */
-//object RunWebLogAnalysis {
-//  def main(args: Array[String]) {
-//    val webLogAnalysis = new WebLogAnalysis
-//    if (args.size < 5) {
-//      println(webLogAnalysis.getDescription)
-//      return
-//    }
-//    val plan = webLogAnalysis.getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4))
-//    LocalExecutor.execute(plan)
-//  }
-//}
-//
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.relational
+
+import org.apache.flink.api.scala._
+import org.apache.flink.example.java.relational.util.WebLogData
+import org.apache.flink.util.Collector
+
+/**
+ * This program processes web logs and relational data.
+ * It implements the following relational query:
+ *
+ * <code><pre>
+ * SELECT
+ *       r.pageURL,
+ *       r.pageRank,
+ *       r.avgDuration
+ * FROM documents d JOIN rankings r
+ *                  ON d.url = r.url
+ * WHERE CONTAINS(d.text, [keywords])
+ *       AND r.rank > [rank]
+ *       AND NOT EXISTS
+ *           (
+ *              SELECT * FROM Visits v
+ *              WHERE v.destUrl = d.url
+ *                    AND v.visitDate < [date]
+ *           );
+ * </pre></code>
+ *
+ * <p>
+ * Input files are plain text CSV files using the pipe character ('|') as field separator.
+ * The tables referenced in the query can be generated using the {@link WebLogDataGenerator} and
+ * have the following schemas
+ * <code><pre>
+ * CREATE TABLE Documents (
+ *                url VARCHAR(100) PRIMARY KEY,
+ *                contents TEXT );
+ *
+ * CREATE TABLE Rankings (
+ *                pageRank INT,
+ *                pageURL VARCHAR(100) PRIMARY KEY,
+ *                avgDuration INT );
+ *
+ * CREATE TABLE Visits (
+ *                sourceIP VARCHAR(16),
+ *                destURL VARCHAR(100),
+ *                visitDate DATE,
+ *                adRevenue FLOAT,
+ *                userAgent VARCHAR(64),
+ *                countryCode VARCHAR(3),
+ *                languageCode VARCHAR(6),
+ *                searchWord VARCHAR(32),
+ *                duration INT );
+ * </pre></code>
+ *
+ * <p>
+ * Usage: <code>WebLogAnalysis &lt;documents path&gt; &lt;ranks path&gt; &lt;visits path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WebLogData}.
+ *
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> tuple data types
+ * <li> projection and join projection
+ * <li> the CoGroup transformation for an anti-join
+ * </ul>
+ *
+ */
+object WebLogAnalysis {
+
+	def main(args: Array[String]) {
+		if (!parseParameters(args)) {
+			return
+		}
+
+		val env = ExecutionEnvironment.getExecutionEnvironment
+
+		val documents = getDocumentsDataSet(env)
+		val ranks = getRanksDataSet(env)
+		val visits = getVisitsDataSet(env)
+
+		val filteredDocs = documents
+			.filter (doc => doc._2.contains(" editors ") && doc._2.contains ( " oscillations "))
+
+		val filteredRanks = ranks
+			.filter (rank => rank._1 > 40)
+
+		val filteredVisits = visits
+			.filter (visit => visit._2.substring(0,4).toInt == 2007)
+
+		val joinDocsRanks = filteredDocs
+			.join(filteredRanks)
+			.where(0).equalTo(1)
+			.map(_._2)
+
+		val result = joinDocsRanks
+			.coGroup(filteredVisits)
+			.where(1).equalTo(0)
+			.apply ((ranks, visits, out:Collector[(Int,String,Int)]) => {
+				if (visits.isEmpty) for (rank <- ranks) out.collect(rank)
+		})
+
+		result.print()
+
+		env.execute("WebLogAnalysis Example")
+	}
+
+
+	private var fileOutput: Boolean = false
+	private var documentsPath: String = null
+	private var ranksPath: String = null
+	private var visitsPath: String = null
+	private var outputPath: String = null
+
+	private def parseParameters (args: Array[String]) : Boolean = {
+		if (args.length > 0) {
+			fileOutput = true;
+			if (args.length == 4) {
+				documentsPath = args(0)
+				ranksPath = args(1)
+				visitsPath = args(2)
+				outputPath = args(3)
+			}
+			else {
+				System.err.println("Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>")
+				return false
+			}
+		}
+		else {
+			System.out.println("Executing WebLog Analysis example with built-in default data.")
+			System.out.println("  Provide parameters to read input data from files.")
+			System.out.println("  See the documentation for the correct format of input files.")
+			System.out.println("  We provide a data generator to create synthetic input files for this program.")
+			System.out.println("  Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>")
+		}
+		return true;
+	}
+
+	private def getDocumentsDataSet(env: ExecutionEnvironment): DataSet[(String,String)] = {
+		if (fileOutput) {
+			env.readCsvFile[(String,String)](
+				documentsPath,
+				fieldDelimiter = '|',
+				includedFields = Array(0,1))
+		}
+		else {
+			val documents = WebLogData.DOCUMENTS map {
+				case Array(x,y) => (x.asInstanceOf[String],y.asInstanceOf[String])
+			}
+			env.fromCollection(documents)
+		}
+	}
+
+	private def getRanksDataSet(env: ExecutionEnvironment) : DataSet[(Int, String, Int)] = {
+		if (fileOutput) {
+			env.readCsvFile[(Int,String,Int)](
+				ranksPath,
+				fieldDelimiter = '|',
+				includedFields = Array(0,1,2))
+		}
+		else {
+			val ranks = WebLogData.RANKS map {
+				case Array(x,y,z) => (x.asInstanceOf[Int], y.asInstanceOf[String], z.asInstanceOf[Int])
+			}
+			env.fromCollection(ranks)
+		}
+	}
+
+	private def getVisitsDataSet (env: ExecutionEnvironment) : DataSet[(String,String)] = {
+		if (fileOutput) {
+			env.readCsvFile[(String,String)](
+				visitsPath,
+				fieldDelimiter = '|',
+				includedFields = Array(0,1))
+		}
+		else {
+			val visits = WebLogData.VISITS map {
+				case Array(x,y) => (x.asInstanceOf[String], y.asInstanceOf[String])
+			}
+			env.fromCollection(visits)
+		}
+	}
+}
\ No newline at end of file


[27/60] git commit: Renamed java examples package

Posted by al...@apache.org.
Renamed java examples package


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

Branch: refs/heads/master
Commit: 4cce46ebd8d77c3db8fcb744ef93a3fcce7b7b1c
Parents: 7fe9273
Author: mbalassi <ba...@gmail.com>
Authored: Sun Sep 14 14:58:01 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../flink/example/java/clustering/KMeans.java   | 337 ---------------
 .../java/clustering/util/KMeansData.java        | 105 -----
 .../clustering/util/KMeansDataGenerator.java    | 182 --------
 .../example/java/graph/ConnectedComponents.java | 244 -----------
 .../example/java/graph/EnumTrianglesBasic.java  | 230 ----------
 .../example/java/graph/EnumTrianglesOpt.java    | 354 ---------------
 .../flink/example/java/graph/PageRankBasic.java | 288 -------------
 .../java/graph/TransitiveClosureNaive.java      | 159 -------
 .../graph/util/ConnectedComponentsData.java     |  77 ----
 .../java/graph/util/EnumTrianglesData.java      |  59 ---
 .../java/graph/util/EnumTrianglesDataTypes.java | 117 -----
 .../example/java/graph/util/PageRankData.java   |  87 ----
 .../flink/example/java/misc/PiEstimation.java   | 108 -----
 .../flink/example/java/ml/LinearRegression.java | 316 --------------
 .../java/ml/util/LinearRegressionData.java      |  72 ----
 .../ml/util/LinearRegressionDataGenerator.java  | 113 -----
 .../relational/EmptyFieldsCountAccumulator.java | 261 -----------
 .../java/relational/RelationalQuery.java        | 171 --------
 .../example/java/relational/TPCHQuery10.java    | 247 -----------
 .../example/java/relational/TPCHQuery3.java     | 298 -------------
 .../example/java/relational/WebLogAnalysis.java | 328 --------------
 .../java/relational/util/WebLogData.java        | 428 -------------------
 .../relational/util/WebLogDataGenerator.java    | 211 ---------
 .../flink/example/java/wordcount/WordCount.java | 147 -------
 .../java/wordcount/util/WordCountData.java      |  72 ----
 .../flink/examples/java/clustering/KMeans.java  | 337 +++++++++++++++
 .../java/clustering/util/KMeansData.java        | 105 +++++
 .../clustering/util/KMeansDataGenerator.java    | 182 ++++++++
 .../java/graph/ConnectedComponents.java         | 244 +++++++++++
 .../examples/java/graph/EnumTrianglesBasic.java | 230 ++++++++++
 .../examples/java/graph/EnumTrianglesOpt.java   | 354 +++++++++++++++
 .../examples/java/graph/PageRankBasic.java      | 288 +++++++++++++
 .../java/graph/TransitiveClosureNaive.java      | 158 +++++++
 .../graph/util/ConnectedComponentsData.java     |  77 ++++
 .../java/graph/util/EnumTrianglesData.java      |  59 +++
 .../java/graph/util/EnumTrianglesDataTypes.java | 117 +++++
 .../examples/java/graph/util/PageRankData.java  |  87 ++++
 .../flink/examples/java/misc/PiEstimation.java  | 108 +++++
 .../examples/java/ml/LinearRegression.java      | 316 ++++++++++++++
 .../java/ml/util/LinearRegressionData.java      |  72 ++++
 .../ml/util/LinearRegressionDataGenerator.java  | 113 +++++
 .../relational/EmptyFieldsCountAccumulator.java | 261 +++++++++++
 .../java/relational/RelationalQuery.java        | 171 ++++++++
 .../examples/java/relational/TPCHQuery10.java   | 247 +++++++++++
 .../examples/java/relational/TPCHQuery3.java    | 298 +++++++++++++
 .../java/relational/WebLogAnalysis.java         | 328 ++++++++++++++
 .../java/relational/util/WebLogData.java        | 428 +++++++++++++++++++
 .../relational/util/WebLogDataGenerator.java    | 211 +++++++++
 .../examples/java/wordcount/WordCount.java      | 147 +++++++
 .../java/wordcount/util/WordCountData.java      |  72 ++++
 .../examples/scala/clustering/KMeans.scala      |   2 +-
 .../scala/graph/ConnectedComponents.scala       | 119 ++----
 .../scala/graph/EnumTrianglesBasic.scala        |   4 +-
 .../examples/scala/graph/EnumTrianglesOpt.scala |   4 +-
 .../examples/scala/graph/PageRankBasic.scala    |   4 +-
 .../scala/graph/TransitiveClosureNaive.scala    | 108 +----
 .../examples/scala/ml/LinearRegression.scala    |   3 +-
 .../scala/relational/WebLogAnalysis.scala       | 270 ++++++------
 .../examples/scala/wordcount/WordCount.scala    |   2 +-
 .../iterations/PageRankCompilerTest.java        |  10 +-
 .../compiler/plandump/DumpCompiledPlanTest.java |   2 +-
 .../ConnectedComponentsITCase.java              |   2 +-
 .../EnumTriangleBasicITCase.java                |   2 +-
 .../EnumTriangleOptITCase.java                  |   2 +-
 .../exampleJavaPrograms/PageRankITCase.java     |   2 +-
 .../TransitiveClosureITCase.java                |   2 +-
 .../WebLogAnalysisITCase.java                   |   2 +-
 .../exampleJavaPrograms/WordCountITCase.java    |   2 +-
 .../WordCountWithCollectionITCase.java          |   2 +-
 .../CoGroupConnectedComponentsSecondITCase.java |   4 +-
 70 files changed, 5229 insertions(+), 5340 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
deleted file mode 100644
index 3bea458..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
+++ /dev/null
@@ -1,337 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.clustering;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.example.java.clustering.util.KMeansData;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-
-/**
- * This example implements a basic K-Means clustering algorithm.
- * 
- * <p>
- * K-Means is an iterative clustering algorithm and works as follows:<br>
- * K-Means is given a set of data points to be clustered and an initial set of <i>K</i> cluster centers.
- * In each iteration, the algorithm computes the distance of each data point to each cluster center.
- * Each point is assigned to the cluster center which is closest to it.
- * Subsequently, each cluster center is moved to the center (<i>mean</i>) of all points that have been assigned to it.
- * The moved cluster centers are fed into the next iteration. 
- * The algorithm terminates after a fixed number of iterations (as in this implementation) 
- * or if cluster centers do not (significantly) move in an iteration.<br>
- * This is the Wikipedia entry for the <a href="http://en.wikipedia.org/wiki/K-means_clustering">K-Means Clustering algorithm</a>.
- * 
- * <p>
- * This implementation works on two-dimensional data points. <br>
- * It computes an assignment of data points to cluster centers, i.e., 
- * each data point is annotated with the id of the final cluster (center) it belongs to.
- * 
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Data points are represented as two double values separated by a blank character.
- * Data points are separated by newline characters.<br>
- * For example <code>"1.2 2.3\n5.3 7.2\n"</code> gives two data points (x=1.2, y=2.3) and (x=5.3, y=7.2).
- * <li>Cluster centers are represented by an integer id and a point value.<br>
- * For example <code>"1 6.2 3.2\n2 2.9 5.7\n"</code> gives two centers (id=1, x=6.2, y=3.2) and (id=2, x=2.9, y=5.7).
- * </ul>
- * 
- * <p>
- * Usage: <code>KMeans &lt;points path&gt; &lt;centers path&gt; &lt;result path&gt; &lt;num iterations&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link KMeansData} and 10 iterations. 
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li>Bulk iterations
- * <li>Broadcast variables in bulk iterations
- * <li>Custom Java objects (PoJos)
- * </ul>
- */
-@SuppressWarnings("serial")
-public class KMeans {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-	
-		// set up execution environment
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// get input data
-		DataSet<Point> points = getPointDataSet(env);
-		DataSet<Centroid> centroids = getCentroidDataSet(env);
-		
-		// set number of bulk iterations for KMeans algorithm
-		IterativeDataSet<Centroid> loop = centroids.iterate(numIterations);
-		
-		DataSet<Centroid> newCentroids = points
-			// compute closest centroid for each point
-			.map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids")
-			// count and sum point coordinates for each centroid
-			.map(new CountAppender())
-			.groupBy(0).reduce(new CentroidAccumulator())
-			// compute new centroids from point counts and coordinate sums
-			.map(new CentroidAverager());
-		
-		// feed new centroids back into next iteration
-		DataSet<Centroid> finalCentroids = loop.closeWith(newCentroids);
-		
-		DataSet<Tuple2<Integer, Point>> clusteredPoints = points
-				// assign points to final clusters
-				.map(new SelectNearestCenter()).withBroadcastSet(finalCentroids, "centroids");
-		
-		// emit result
-		if(fileOutput) {
-			clusteredPoints.writeAsCsv(outputPath, "\n", " ");
-		} else {
-			clusteredPoints.print();
-		}
-
-		// execute program
-		env.execute("KMeans Example");
-		
-	}
-	
-	// *************************************************************************
-	//     DATA TYPES
-	// *************************************************************************
-	
-	/**
-	 * A simple two-dimensional point.
-	 */
-	public static class Point implements Serializable {
-		
-		public double x, y;
-		
-		public Point() {}
-
-		public Point(double x, double y) {
-			this.x = x;
-			this.y = y;
-		}
-		
-		public Point add(Point other) {
-			x += other.x;
-			y += other.y;
-			return this;
-		}
-		
-		public Point div(long val) {
-			x /= val;
-			y /= val;
-			return this;
-		}
-		
-		public double euclideanDistance(Point other) {
-			return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y));
-		}
-		
-		public void clear() {
-			x = y = 0.0;
-		}
-		
-		@Override
-		public String toString() {
-			return x + " " + y;
-		}
-	}
-	
-	/**
-	 * A simple two-dimensional centroid, basically a point with an ID. 
-	 */
-	public static class Centroid extends Point {
-		
-		public int id;
-		
-		public Centroid() {}
-		
-		public Centroid(int id, double x, double y) {
-			super(x,y);
-			this.id = id;
-		}
-		
-		public Centroid(int id, Point p) {
-			super(p.x, p.y);
-			this.id = id;
-		}
-		
-		@Override
-		public String toString() {
-			return id + " " + super.toString();
-		}
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-	
-	/** Converts a Tuple2<Double,Double> into a Point. */
-	public static final class TuplePointConverter implements MapFunction<Tuple2<Double, Double>, Point> {
-
-		@Override
-		public Point map(Tuple2<Double, Double> t) throws Exception {
-			return new Point(t.f0, t.f1);
-		}
-	}
-	
-	/** Converts a Tuple3<Integer, Double,Double> into a Centroid. */
-	public static final class TupleCentroidConverter implements MapFunction<Tuple3<Integer, Double, Double>, Centroid> {
-
-		@Override
-		public Centroid map(Tuple3<Integer, Double, Double> t) throws Exception {
-			return new Centroid(t.f0, t.f1, t.f2);
-		}
-	}
-	
-	/** Determines the closest cluster center for a data point. */
-	public static final class SelectNearestCenter extends RichMapFunction<Point, Tuple2<Integer, Point>> {
-		private Collection<Centroid> centroids;
-
-		/** Reads the centroid values from a broadcast variable into a collection. */
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			this.centroids = getRuntimeContext().getBroadcastVariable("centroids");
-		}
-		
-		@Override
-		public Tuple2<Integer, Point> map(Point p) throws Exception {
-			
-			double minDistance = Double.MAX_VALUE;
-			int closestCentroidId = -1;
-			
-			// check all cluster centers
-			for (Centroid centroid : centroids) {
-				// compute distance
-				double distance = p.euclideanDistance(centroid);
-				
-				// update nearest cluster if necessary 
-				if (distance < minDistance) {
-					minDistance = distance;
-					closestCentroidId = centroid.id;
-				}
-			}
-
-			// emit a new record with the center id and the data point.
-			return new Tuple2<Integer, Point>(closestCentroidId, p);
-		}
-	}
-	
-	/** Appends a count variable to the tuple. */ 
-	public static final class CountAppender implements MapFunction<Tuple2<Integer, Point>, Tuple3<Integer, Point, Long>> {
-
-		@Override
-		public Tuple3<Integer, Point, Long> map(Tuple2<Integer, Point> t) {
-			return new Tuple3<Integer, Point, Long>(t.f0, t.f1, 1L);
-		} 
-	}
-	
-	/** Sums and counts point coordinates. */
-	public static final class CentroidAccumulator implements ReduceFunction<Tuple3<Integer, Point, Long>> {
-
-		@Override
-		public Tuple3<Integer, Point, Long> reduce(Tuple3<Integer, Point, Long> val1, Tuple3<Integer, Point, Long> val2) {
-			return new Tuple3<Integer, Point, Long>(val1.f0, val1.f1.add(val2.f1), val1.f2 + val2.f2);
-		}
-	}
-	
-	/** Computes new centroid from coordinate sum and count of points. */
-	public static final class CentroidAverager implements MapFunction<Tuple3<Integer, Point, Long>, Centroid> {
-
-		@Override
-		public Centroid map(Tuple3<Integer, Point, Long> value) {
-			return new Centroid(value.f0, value.f1.div(value.f2));
-		}
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String pointsPath = null;
-	private static String centersPath = null;
-	private static String outputPath = null;
-	private static int numIterations = 10;
-	
-	private static boolean parseParameters(String[] programArguments) {
-		
-		if(programArguments.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(programArguments.length == 4) {
-				pointsPath = programArguments[0];
-				centersPath = programArguments[1];
-				outputPath = programArguments[2];
-				numIterations = Integer.parseInt(programArguments[3]);
-			} else {
-				System.err.println("Usage: KMeans <points path> <centers path> <result path> <num iterations>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing K-Means example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  We provide a data generator to create synthetic input files for this program.");
-			System.out.println("  Usage: KMeans <points path> <centers path> <result path> <num iterations>");
-		}
-		return true;
-	}
-	
-	private static DataSet<Point> getPointDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			// read points from CSV file
-			return env.readCsvFile(pointsPath)
-						.fieldDelimiter(' ')
-						.includeFields(true, true)
-						.types(Double.class, Double.class)
-						.map(new TuplePointConverter());
-		} else {
-			return KMeansData.getDefaultPointDataSet(env);
-		}
-	}
-	
-	private static DataSet<Centroid> getCentroidDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return env.readCsvFile(centersPath)
-						.fieldDelimiter(' ')
-						.includeFields(true, true, true)
-						.types(Integer.class, Double.class, Double.class)
-						.map(new TupleCentroidConverter());
-		} else {
-			return KMeansData.getDefaultCentroidDataSet(env);
-		}
-	}
-		
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
deleted file mode 100644
index 97325dc..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.clustering.util;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.clustering.KMeans.Centroid;
-import org.apache.flink.example.java.clustering.KMeans.Point;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the K-Means example program.
- * The default data sets are used, if no parameters are given to the program.
- *
- */
-public class KMeansData {
-
-	// We have the data as object arrays so that we can also generate Scala Data Sources from it.
-	public static final Object[][] CENTROIDS = new Object[][] {
-		new Object[] {1, -31.85, -44.77},
-		new Object[]{2, 35.16, 17.46},
-		new Object[]{3, -5.16, 21.93},
-		new Object[]{4, -24.06, 6.81}
-	};
-
-	public static final Object[][] POINTS = new Object[][] {
-		new Object[] {-14.22, -48.01},
-		new Object[] {-22.78, 37.10},
-		new Object[] {56.18, -42.99},
-		new Object[] {35.04, 50.29},
-		new Object[] {-9.53, -46.26},
-		new Object[] {-34.35, 48.25},
-		new Object[] {55.82, -57.49},
-		new Object[] {21.03, 54.64},
-		new Object[] {-13.63, -42.26},
-		new Object[] {-36.57, 32.63},
-		new Object[] {50.65, -52.40},
-		new Object[] {24.48, 34.04},
-		new Object[] {-2.69, -36.02},
-		new Object[] {-38.80, 36.58},
-		new Object[] {24.00, -53.74},
-		new Object[] {32.41, 24.96},
-		new Object[] {-4.32, -56.92},
-		new Object[] {-22.68, 29.42},
-		new Object[] {59.02, -39.56},
-		new Object[] {24.47, 45.07},
-		new Object[] {5.23, -41.20},
-		new Object[] {-23.00, 38.15},
-		new Object[] {44.55, -51.50},
-		new Object[] {14.62, 59.06},
-		new Object[] {7.41, -56.05},
-		new Object[] {-26.63, 28.97},
-		new Object[] {47.37, -44.72},
-		new Object[] {29.07, 51.06},
-		new Object[] {0.59, -31.89},
-		new Object[] {-39.09, 20.78},
-		new Object[] {42.97, -48.98},
-		new Object[] {34.36, 49.08},
-		new Object[] {-21.91, -49.01},
-		new Object[] {-46.68, 46.04},
-		new Object[] {48.52, -43.67},
-		new Object[] {30.05, 49.25},
-		new Object[] {4.03, -43.56},
-		new Object[] {-37.85, 41.72},
-		new Object[] {38.24, -48.32},
-		new Object[] {20.83, 57.85}
-	};
-
-	public static DataSet<Centroid> getDefaultCentroidDataSet(ExecutionEnvironment env) {
-		List<Centroid> centroidList = new LinkedList<Centroid>();
-		for (Object[] centroid : CENTROIDS) {
-			centroidList.add(
-					new Centroid((Integer) centroid[0], (Double) centroid[1], (Double) centroid[2]));
-		}
-		return env.fromCollection(centroidList);
-	}
-	
-	public static DataSet<Point> getDefaultPointDataSet(ExecutionEnvironment env) {
-		List<Point> pointList = new LinkedList<Point>();
-		for (Object[] point : POINTS) {
-			pointList.add(new Point((Double) point[0], (Double) point[1]));
-		}
-		return env.fromCollection(pointList);
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansDataGenerator.java
deleted file mode 100644
index c1d47da..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansDataGenerator.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.clustering.util;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.text.DecimalFormat;
-import java.util.Locale;
-import java.util.Random;
-
-import org.apache.flink.example.java.clustering.KMeans;
-
-/**
- * Generates data for the {@link KMeans} example program.
- */
-public class KMeansDataGenerator {
-	
-	static {
-		Locale.setDefault(Locale.US);
-	}
-	
-	private static final String CENTERS_FILE = "centers";
-	private static final String POINTS_FILE = "points";
-	private static final long DEFAULT_SEED = 4650285087650871364L;
-	private static final double DEFAULT_VALUE_RANGE = 100.0;
-	private static final double RELATIVE_STDDEV = 0.08;
-	private static final int DIMENSIONALITY = 2;
-	private static final DecimalFormat FORMAT = new DecimalFormat("#0.00");
-	private static final char DELIMITER = ' ';
-
-	/**
-	 * Main method to generate data for the {@link KMeans} example program.
-	 * <p>
-	 * The generator creates to files:
-	 * <ul>
-	 * <li><code>{tmp.dir}/points</code> for the data points
-	 * <li><code>{tmp.dir}/centers</code> for the cluster centers
-	 * </ul> 
-	 * 
-	 * @param args 
-	 * <ol>
-	 * <li>Int: Number of data points
-	 * <li>Int: Number of cluster centers
-	 * <li><b>Optional</b> Double: Standard deviation of data points
-	 * <li><b>Optional</b> Double: Value range of cluster centers
-	 * <li><b>Optional</b> Long: Random seed
-	 * </ol>
-	 */
-	public static void main(String[] args) throws IOException {
-
-		// check parameter count
-		if (args.length < 2) {
-			System.out.println("KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]");
-			System.exit(1);
-		}
-
-		// parse parameters
-		final int numDataPoints = Integer.parseInt(args[0]);
-		final int k = Integer.parseInt(args[1]);
-		final double stddev = args.length > 2 ? Double.parseDouble(args[2]) : RELATIVE_STDDEV;
-		final double range = args.length > 3 ? Double.parseDouble(args[4]) : DEFAULT_VALUE_RANGE;
-		final long firstSeed = args.length > 4 ? Long.parseLong(args[4]) : DEFAULT_SEED;
-		
-		final double absoluteStdDev = stddev * range;
-		final Random random = new Random(firstSeed);
-		final String tmpDir = System.getProperty("java.io.tmpdir");
-		
-		// the means around which data points are distributed
-		final double[][] means = uniformRandomCenters(random, k, DIMENSIONALITY, range);
-		
-		// write the points out
-		BufferedWriter pointsOut = null;
-		try {
-			pointsOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+POINTS_FILE)));
-			StringBuilder buffer = new StringBuilder();
-			
-			double[] point = new double[DIMENSIONALITY];
-			int nextCentroid = 0;
-			
-			for (int i = 1; i <= numDataPoints; i++) {
-				// generate a point for the current centroid
-				double[] centroid = means[nextCentroid];
-				for (int d = 0; d < DIMENSIONALITY; d++) {
-					point[d] = (random.nextGaussian() * absoluteStdDev) + centroid[d];
-				}
-				writePoint(point, buffer, pointsOut);
-				nextCentroid = (nextCentroid + 1) % k;
-			}
-		}
-		finally {
-			if (pointsOut != null) {
-				pointsOut.close();
-			}
-		}
-		
-		// write the uniformly distributed centers to a file
-		BufferedWriter centersOut = null;
-		try {
-			centersOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+CENTERS_FILE)));
-			StringBuilder buffer = new StringBuilder();
-			
-			double[][] centers = uniformRandomCenters(random, k, DIMENSIONALITY, range);
-			
-			for (int i = 0; i < k; i++) {
-				writeCenter(i + 1, centers[i], buffer, centersOut);
-			}
-		}
-		finally {
-			if (centersOut != null) {
-				centersOut.close();
-			}
-		}
-		
-		System.out.println("Wrote "+numDataPoints+" data points to "+tmpDir+"/"+POINTS_FILE);
-		System.out.println("Wrote "+k+" cluster centers to "+tmpDir+"/"+CENTERS_FILE);
-	}
-	
-	private static final double[][] uniformRandomCenters(Random rnd, int num, int dimensionality, double range) {
-		final double halfRange = range / 2;
-		final double[][] points = new double[num][dimensionality];
-		
-		for (int i = 0; i < num; i++) {
-			for (int dim = 0; dim < dimensionality; dim ++) {
-				points[i][dim] = (rnd.nextDouble() * range) - halfRange;
-			}
-		}
-		return points;
-	}
-	
-	private static void writePoint(double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException {
-		buffer.setLength(0);
-		
-		// write coordinates
-		for (int j = 0; j < coordinates.length; j++) {
-			buffer.append(FORMAT.format(coordinates[j]));
-			if(j < coordinates.length - 1) {
-				buffer.append(DELIMITER);
-			}
-		}
-		
-		out.write(buffer.toString());
-		out.newLine();
-	}
-	
-	private static void writeCenter(long id, double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException {
-		buffer.setLength(0);
-		
-		// write id
-		buffer.append(id);
-		buffer.append(DELIMITER);
-
-		// write coordinates
-		for (int j = 0; j < coordinates.length; j++) {
-			buffer.append(FORMAT.format(coordinates[j]));
-			if(j < coordinates.length - 1) {
-				buffer.append(DELIMITER);
-			}
-		}
-		
-		out.write(buffer.toString());
-		out.newLine();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
deleted file mode 100644
index 93e8baa..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.graph;
-
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.graph.util.ConnectedComponentsData;
-
-/**
- * An implementation of the connected components algorithm, using a delta iteration.
- * 
- * <p>
- * Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the minimum of its own ID and its
- * neighbors' IDs, as its new ID and tells its neighbors about its new ID. After the algorithm has completed, all vertices in the
- * same component will have the same ID.
- * 
- * <p>
- * A vertex whose component ID did not change needs not propagate its information in the next step. Because of that,
- * the algorithm is easily expressible via a delta iteration. We here model the solution set as the vertices with
- * their current component ids, and the workset as the changed vertices. Because we see all vertices initially as
- * changed, the initial workset and the initial solution set are identical. Also, the delta to the solution set
- * is consequently also the next workset.<br>
- * 
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Vertices represented as IDs and separated by new-line characters.<br> 
- * For example <code>"1\n2\n12\n42\n63\n"</code> gives five vertices (1), (2), (12), (42), and (63). 
- * <li>Edges are represented as pairs for vertex IDs which are separated by space 
- * characters. Edges are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
- * </ul>
- * 
- * <p>
- * Usage: <code>ConnectedComponents &lt;vertices path&gt; &lt;edges path&gt; &lt;result path&gt; &lt;max number of iterations&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link ConnectedComponentsData} and 10 iterations. 
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li>Delta Iterations
- * <li>Generic-typed Functions 
- * </ul>
- */
-@SuppressWarnings("serial")
-public class ConnectedComponents implements ProgramDescription {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String... args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		// set up execution environment
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// read vertex and edge data
-		DataSet<Long> vertices = getVertexDataSet(env);
-		DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
-		
-		// assign the initial components (equal to the vertex id)
-		DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
-				
-		// open a delta iteration
-		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
-				verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0);
-		
-		// apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller
-		DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset().join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
-				.groupBy(0).aggregate(Aggregations.MIN, 1)
-				.join(iteration.getSolutionSet()).where(0).equalTo(0)
-				.with(new ComponentIdFilter());
-
-		// close the delta iteration (delta and new workset are identical)
-		DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
-		
-		// emit result
-		if(fileOutput) {
-			result.writeAsCsv(outputPath, "\n", " ");
-		} else {
-			result.print();
-		}
-		
-		// execute program
-		env.execute("Connected Components Example");
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-	
-	/**
-	 * Function that turns a value into a 2-tuple where both fields are that value.
-	 */
-	@ConstantFields("0 -> 0,1") 
-	public static final class DuplicateValue<T> implements MapFunction<T, Tuple2<T, T>> {
-		
-		@Override
-		public Tuple2<T, T> map(T vertex) {
-			return new Tuple2<T, T>(vertex, vertex);
-		}
-	}
-	
-	/**
-	 * Undirected edges by emitting for each input edge the input edges itself and an inverted version.
-	 */
-	public static final class UndirectEdge implements FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-		Tuple2<Long, Long> invertedEdge = new Tuple2<Long, Long>();
-		
-		@Override
-		public void flatMap(Tuple2<Long, Long> edge, Collector<Tuple2<Long, Long>> out) {
-			invertedEdge.f0 = edge.f1;
-			invertedEdge.f1 = edge.f0;
-			out.collect(edge);
-			out.collect(invertedEdge);
-		}
-	}
-	
-	/**
-	 * UDF that joins a (Vertex-ID, Component-ID) pair that represents the current component that
-	 * a vertex is associated with, with a (Source-Vertex-ID, Target-VertexID) edge. The function
-	 * produces a (Target-vertex-ID, Component-ID) pair.
-	 */
-	@ConstantFieldsFirst("1 -> 0")
-	@ConstantFieldsSecond("1 -> 1")
-	public static final class NeighborWithComponentIDJoin implements JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		@Override
-		public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithComponent, Tuple2<Long, Long> edge) {
-			return new Tuple2<Long, Long>(edge.f1, vertexWithComponent.f1);
-		}
-	}
-	
-
-
-	@ConstantFieldsFirst("0")
-	public static final class ComponentIdFilter implements FlatJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		@Override
-		public void join(Tuple2<Long, Long> candidate, Tuple2<Long, Long> old, Collector<Tuple2<Long, Long>> out) {
-			if (candidate.f1 < old.f1) {
-				out.collect(candidate);
-			}
-		}
-	}
-
-
-
-	@Override
-	public String getDescription() {
-		return "Parameters: <vertices-path> <edges-path> <result-path> <max-number-of-iterations>";
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String verticesPath = null;
-	private static String edgesPath = null;
-	private static String outputPath = null;
-	private static int maxIterations = 10;
-	
-	private static boolean parseParameters(String[] programArguments) {
-		
-		if(programArguments.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(programArguments.length == 4) {
-				verticesPath = programArguments[0];
-				edgesPath = programArguments[1];
-				outputPath = programArguments[2];
-				maxIterations = Integer.parseInt(programArguments[3]);
-			} else {
-				System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing Connected Components example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>");
-		}
-		return true;
-	}
-	
-	private static DataSet<Long> getVertexDataSet(ExecutionEnvironment env) {
-		
-		if(fileOutput) {
-			return env.readCsvFile(verticesPath).types(Long.class)
-						.map(
-								new MapFunction<Tuple1<Long>, Long>() {
-									public Long map(Tuple1<Long> value) { return value.f0; }
-								});
-		} else {
-			return ConnectedComponentsData.getDefaultVertexDataSet(env);
-		}
-	}
-	
-	private static DataSet<Tuple2<Long, Long>> getEdgeDataSet(ExecutionEnvironment env) {
-		
-		if(fileOutput) {
-			return env.readCsvFile(edgesPath).fieldDelimiter(' ').types(Long.class, Long.class); 
-		} else {
-			return ConnectedComponentsData.getDefaultEdgeDataSet(env);
-		}
-	}
-	
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java
deleted file mode 100644
index 3d68e99..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesBasic.java
+++ /dev/null
@@ -1,230 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.graph;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.graph.util.EnumTrianglesData;
-import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Edge;
-import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Triad;
-
-/**
- * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
- * A triangle consists of three edges that connect three vertices with each other.
- * 
- * <p>
- * The algorithm works as follows: 
- * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
- * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
- * that closes the triangle.
- *  
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Edges are represented as pairs for vertex IDs which are separated by space 
- * characters. Edges are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
- * that include a triangle
- * </ul>
- * <pre>
- *     (1)
- *     /  \
- *   (2)-(12)
- * </pre>
- * 
- * Usage: <code>EnumTriangleBasic &lt;edge path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link EnumTrianglesData}. 
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li>Custom Java objects which extend Tuple
- * <li>Group Sorting
- * </ul>
- * 
- */
-@SuppressWarnings("serial")
-public class EnumTrianglesBasic {
-
-	static boolean fileOutput = false;
-	static String edgePath = null;
-	static String outputPath = null;
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		// set up execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-	
-		// read input data
-		DataSet<Edge> edges = getEdgeDataSet(env);
-		
-		// project edges by vertex id
-		DataSet<Edge> edgesById = edges
-				.map(new EdgeByIdProjector());
-		
-		DataSet<Triad> triangles = edgesById
-				// build triads
-				.groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder())
-				// filter triads
-				.join(edgesById).where(Triad.V2, Triad.V3).equalTo(Edge.V1, Edge.V2).with(new TriadFilter());
-
-		// emit result
-		if(fileOutput) {
-			triangles.writeAsCsv(outputPath, "\n", ",");
-		} else {
-			triangles.print();
-		}
-
-		// execute program
-		env.execute("Basic Triangle Enumeration Example");
-
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-
-	/** Converts a Tuple2 into an Edge */
-	public static class TupleEdgeConverter implements MapFunction<Tuple2<Integer, Integer>, Edge> {
-		private final Edge outEdge = new Edge();
-		
-		@Override
-		public Edge map(Tuple2<Integer, Integer> t) throws Exception {
-			outEdge.copyVerticesFromTuple2(t);
-			return outEdge;
-		}
-	}
-	
-	/** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */
-	private static class EdgeByIdProjector implements MapFunction<Edge, Edge> {
-	
-		@Override
-		public Edge map(Edge inEdge) throws Exception {
-			
-			// flip vertices if necessary
-			if(inEdge.getFirstVertex() > inEdge.getSecondVertex()) {
-				inEdge.flipVertices();
-			}
-			
-			return inEdge;
-		}
-	}
-	
-	/**
-	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
-	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
-	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
-	 */
-	private static class TriadBuilder implements GroupReduceFunction<Edge, Triad> {
-		private final List<Integer> vertices = new ArrayList<Integer>();
-		private final Triad outTriad = new Triad();
-		
-		@Override
-		public void reduce(Iterable<Edge> edgesIter, Collector<Triad> out) throws Exception {
-			
-			final Iterator<Edge> edges = edgesIter.iterator();
-			
-			// clear vertex list
-			vertices.clear();
-			
-			// read first edge
-			Edge firstEdge = edges.next();
-			outTriad.setFirstVertex(firstEdge.getFirstVertex());
-			vertices.add(firstEdge.getSecondVertex());
-			
-			// build and emit triads
-			while (edges.hasNext()) {
-				Integer higherVertexId = edges.next().getSecondVertex();
-				
-				// combine vertex with all previously read vertices
-				for (Integer lowerVertexId : vertices) {
-					outTriad.setSecondVertex(lowerVertexId);
-					outTriad.setThirdVertex(higherVertexId);
-					out.collect(outTriad);
-				}
-				vertices.add(higherVertexId);
-			}
-		}
-	}
-	
-	/** Filters triads (three vertices connected by two edges) without a closing third edge. */
-	private static class TriadFilter implements JoinFunction<Triad, Edge, Triad> {
-		
-		@Override
-		public Triad join(Triad triad, Edge edge) throws Exception {
-			return triad;
-		}
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean parseParameters(String[] args) {
-	
-		if(args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(args.length == 2) {
-				edgePath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing Enum Triangles Basic example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>");
-		}
-		return true;
-	}
-	
-	private static DataSet<Edge> getEdgeDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return env.readCsvFile(edgePath)
-						.fieldDelimiter(' ')
-						.includeFields(true, true)
-						.types(Integer.class, Integer.class)
-						.map(new TupleEdgeConverter());
-		} else {
-			return EnumTrianglesData.getDefaultEdgeDataSet(env);
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java
deleted file mode 100644
index efccb59..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/EnumTrianglesOpt.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.graph;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.graph.util.EnumTrianglesData;
-import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Edge;
-import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.EdgeWithDegrees;
-import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Triad;
-
-/**
- * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
- * A triangle consists of three edges that connect three vertices with each other.
- * 
- * <p>
- * The basic algorithm works as follows: 
- * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
- * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
- * that closes the triangle.
- * 
- * <p>
- * For a group of <i>n</i> edges that share a common vertex, the number of built triads is quadratic <i>((n*(n-1))/2)</i>.
- * Therefore, an optimization of the algorithm is to group edges on the vertex with the smaller output degree to 
- * reduce the number of triads. 
- * This implementation extends the basic algorithm by computing output degrees of edge vertices and 
- * grouping on edges on the vertex with the smaller degree.
- * 
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Edges are represented as pairs for vertex IDs which are separated by space 
- * characters. Edges are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
- * that include a triangle
- * </ul>
- * <pre>
- *     (1)
- *     /  \
- *   (2)-(12)
- * </pre>
- * 
- * Usage: <code>EnumTriangleOpt &lt;edge path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link EnumTrianglesData}.
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li>Custom Java objects which extend Tuple
- * <li>Group Sorting
- * </ul>
- * 
- */
-@SuppressWarnings("serial")
-public class EnumTrianglesOpt {
-
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		// set up execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// read input data
-		DataSet<Edge> edges = getEdgeDataSet(env);
-		
-		// annotate edges with degrees
-		DataSet<EdgeWithDegrees> edgesWithDegrees = edges
-				.flatMap(new EdgeDuplicator())
-				.groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new DegreeCounter())
-				.groupBy(EdgeWithDegrees.V1,EdgeWithDegrees.V2).reduce(new DegreeJoiner());
-		
-		// project edges by degrees
-		DataSet<Edge> edgesByDegree = edgesWithDegrees
-				.map(new EdgeByDegreeProjector());
-		// project edges by vertex id
-		DataSet<Edge> edgesById = edgesByDegree
-				.map(new EdgeByIdProjector());
-		
-		DataSet<Triad> triangles = edgesByDegree
-				// build triads
-				.groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder())
-				// filter triads
-				.join(edgesById).where(Triad.V2,Triad.V3).equalTo(Edge.V1,Edge.V2).with(new TriadFilter());
-
-		// emit result
-		if(fileOutput) {
-			triangles.writeAsCsv(outputPath, "\n", ",");
-		} else {
-			triangles.print();
-		}
-		
-		// execute program
-		env.execute("Triangle Enumeration Example");
-		
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-	
-	/** Converts a Tuple2 into an Edge */
-	public static class TupleEdgeConverter implements MapFunction<Tuple2<Integer, Integer>, Edge> {
-		private final Edge outEdge = new Edge();
-		
-		@Override
-		public Edge map(Tuple2<Integer, Integer> t) throws Exception {
-			outEdge.copyVerticesFromTuple2(t);
-			return outEdge;
-		}
-	}
-	
-	/** Emits for an edge the original edge and its switched version. */
-	private static class EdgeDuplicator implements FlatMapFunction<Edge, Edge> {
-		
-		@Override
-		public void flatMap(Edge edge, Collector<Edge> out) throws Exception {
-			out.collect(edge);
-			edge.flipVertices();
-			out.collect(edge);
-		}
-	}
-	
-	/**
-	 * Counts the number of edges that share a common vertex.
-	 * Emits one edge for each input edge with a degree annotation for the shared vertex.
-	 * For each emitted edge, the first vertex is the vertex with the smaller id.
-	 */
-	private static class DegreeCounter implements GroupReduceFunction<Edge, EdgeWithDegrees> {
-		
-		final ArrayList<Integer> otherVertices = new ArrayList<Integer>();
-		final EdgeWithDegrees outputEdge = new EdgeWithDegrees();
-		
-		@Override
-		public void reduce(Iterable<Edge> edgesIter, Collector<EdgeWithDegrees> out) {
-			
-			Iterator<Edge> edges = edgesIter.iterator();
-			otherVertices.clear();
-			
-			// get first edge
-			Edge edge = edges.next();
-			Integer groupVertex = edge.getFirstVertex();
-			this.otherVertices.add(edge.getSecondVertex());
-			
-			// get all other edges (assumes edges are sorted by second vertex)
-			while (edges.hasNext()) {
-				edge = edges.next();
-				Integer otherVertex = edge.getSecondVertex();
-				// collect unique vertices
-				if(!otherVertices.contains(otherVertex) && otherVertex != groupVertex) {
-					this.otherVertices.add(otherVertex);
-				}
-			}
-			int degree = this.otherVertices.size();
-			
-			// emit edges
-			for(Integer otherVertex : this.otherVertices) {
-				if(groupVertex < otherVertex) {
-					outputEdge.setFirstVertex(groupVertex);
-					outputEdge.setFirstDegree(degree);
-					outputEdge.setSecondVertex(otherVertex);
-					outputEdge.setSecondDegree(0);
-				} else {
-					outputEdge.setFirstVertex(otherVertex);
-					outputEdge.setFirstDegree(0);
-					outputEdge.setSecondVertex(groupVertex);
-					outputEdge.setSecondDegree(degree);
-				}
-				out.collect(outputEdge);
-			}
-		}
-	}
-	
-	/**
-	 * Builds an edge with degree annotation from two edges that have the same vertices and only one 
-	 * degree annotation.
-	 */
-	private static class DegreeJoiner implements ReduceFunction<EdgeWithDegrees> {
-		private final EdgeWithDegrees outEdge = new EdgeWithDegrees();
-		
-		@Override
-		public EdgeWithDegrees reduce(EdgeWithDegrees edge1, EdgeWithDegrees edge2) throws Exception {
-			
-			// copy first edge
-			outEdge.copyFrom(edge1);
-			
-			// set missing degree
-			if(edge1.getFirstDegree() == 0 && edge1.getSecondDegree() != 0) {
-				outEdge.setFirstDegree(edge2.getFirstDegree());
-			} else if (edge1.getFirstDegree() != 0 && edge1.getSecondDegree() == 0) {
-				outEdge.setSecondDegree(edge2.getSecondDegree());
-			}
-			return outEdge;
-		}
-	}
-		
-	/** Projects an edge (pair of vertices) such that the first vertex is the vertex with the smaller degree. */
-	private static class EdgeByDegreeProjector implements MapFunction<EdgeWithDegrees, Edge> {
-		
-		private final Edge outEdge = new Edge();
-		
-		@Override
-		public Edge map(EdgeWithDegrees inEdge) throws Exception {
-
-			// copy vertices to simple edge
-			outEdge.copyVerticesFromEdgeWithDegrees(inEdge);
-
-			// flip vertices if first degree is larger than second degree.
-			if(inEdge.getFirstDegree() > inEdge.getSecondDegree()) {
-				outEdge.flipVertices();
-			}
-
-			// return edge
-			return outEdge;
-		}
-	}
-	
-	/** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */
-	private static class EdgeByIdProjector implements MapFunction<Edge, Edge> {
-	
-		@Override
-		public Edge map(Edge inEdge) throws Exception {
-			
-			// flip vertices if necessary
-			if(inEdge.getFirstVertex() > inEdge.getSecondVertex()) {
-				inEdge.flipVertices();
-			}
-			
-			return inEdge;
-		}
-	}
-	
-	/**
-	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
-	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
-	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
-	 */
-	private static class TriadBuilder implements GroupReduceFunction<Edge, Triad> {
-		
-		private final List<Integer> vertices = new ArrayList<Integer>();
-		private final Triad outTriad = new Triad();
-		
-		@Override
-		public void reduce(Iterable<Edge> edgesIter, Collector<Triad> out) throws Exception {
-			final Iterator<Edge> edges = edgesIter.iterator();
-			
-			// clear vertex list
-			vertices.clear();
-			
-			// read first edge
-			Edge firstEdge = edges.next();
-			outTriad.setFirstVertex(firstEdge.getFirstVertex());
-			vertices.add(firstEdge.getSecondVertex());
-			
-			// build and emit triads
-			while (edges.hasNext()) {
-				Integer higherVertexId = edges.next().getSecondVertex();
-				
-				// combine vertex with all previously read vertices
-				for(Integer lowerVertexId : vertices) {
-					outTriad.setSecondVertex(lowerVertexId);
-					outTriad.setThirdVertex(higherVertexId);
-					out.collect(outTriad);
-				}
-				vertices.add(higherVertexId);
-			}
-		}
-	}
-	
-	/** Filters triads (three vertices connected by two edges) without a closing third edge. */
-	private static class TriadFilter implements JoinFunction<Triad, Edge, Triad> {
-		
-		@Override
-		public Triad join(Triad triad, Edge edge) throws Exception {
-			return triad;
-		}
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String edgePath = null;
-	private static String outputPath = null;
-	
-	private static boolean parseParameters(String[] args) {
-		
-		if(args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(args.length == 2) {
-				edgePath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing Enum Triangles Opt example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: EnumTriangleOpt <edge path> <result path>");
-		}
-		return true;
-	}
-	
-	private static DataSet<Edge> getEdgeDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return env.readCsvFile(edgePath)
-						.fieldDelimiter(' ')
-						.includeFields(true, true)
-						.types(Integer.class, Integer.class)
-						.map(new TupleEdgeConverter());
-		} else {
-			return EnumTrianglesData.getDefaultEdgeDataSet(env);
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
deleted file mode 100644
index 8a57007..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
+++ /dev/null
@@ -1,288 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.graph;
-
-import static org.apache.flink.api.java.aggregation.Aggregations.SUM;
-
-import java.util.ArrayList;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.example.java.graph.util.PageRankData;
-
-/**
- * A basic implementation of the Page Rank algorithm using a bulk iteration.
- * 
- * <p>
- * This implementation requires a set of pages and a set of directed links as input and works as follows. <br> 
- * In each iteration, the rank of every page is evenly distributed to all pages it points to.
- * Each page collects the partial ranks of all pages that point to it, sums them up, and applies a dampening factor to the sum.
- * The result is the new rank of the page. A new iteration is started with the new ranks of all pages.
- * This implementation terminates after a fixed number of iterations.<br>
- * This is the Wikipedia entry for the <a href="http://en.wikipedia.org/wiki/Page_rank">Page Rank algorithm</a>. 
- * 
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Pages represented as an (long) ID separated by new-line characters.<br> 
- * For example <code>"1\n2\n12\n42\n63\n"</code> gives five pages with IDs 1, 2, 12, 42, and 63.
- * <li>Links are represented as pairs of page IDs which are separated by space 
- * characters. Links are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).<br>
- * For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
- * </ul>
- * 
- * <p>
- * Usage: <code>PageRankBasic &lt;pages path&gt; &lt;links path&gt; &lt;output path&gt; &lt;num pages&gt; &lt;num iterations&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link PageRankData} and 10 iterations.
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li>Bulk Iterations
- * <li>Default Join
- * <li>Configure user-defined functions using constructor parameters.
- * </ul> 
- * 
- *
- */
-@SuppressWarnings("serial")
-public class PageRankBasic {
-	
-	private static final double DAMPENING_FACTOR = 0.85;
-	private static final double EPSILON = 0.0001;
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		// set up execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// get input data
-		DataSet<Long> pagesInput = getPagesDataSet(env);
-		DataSet<Tuple2<Long, Long>> linksInput = getLinksDataSet(env);
-		
-		// assign initial rank to pages
-		DataSet<Tuple2<Long, Double>> pagesWithRanks = pagesInput.
-				map(new RankAssigner((1.0d / numPages)));
-		
-		// build adjacency list from link input
-		DataSet<Tuple2<Long, Long[]>> adjacencyListInput = 
-				linksInput.groupBy(0).reduceGroup(new BuildOutgoingEdgeList());
-		
-		// set iterative data set
-		IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(maxIterations);
-		
-		DataSet<Tuple2<Long, Double>> newRanks = iteration
-				// join pages with outgoing edges and distribute rank
-				.join(adjacencyListInput).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch())
-				// collect and sum ranks
-				.groupBy(0).aggregate(SUM, 1)
-				// apply dampening factor
-				.map(new Dampener(DAMPENING_FACTOR, numPages));
-		
-		DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
-				newRanks, 
-				newRanks.join(iteration).where(0).equalTo(0)
-				// termination condition
-				.filter(new EpsilonFilter()));
-
-		// emit result
-		if(fileOutput) {
-			finalPageRanks.writeAsCsv(outputPath, "\n", " ");
-		} else {
-			finalPageRanks.print();
-		}
-
-		// execute program
-		env.execute("Basic Page Rank Example");
-		
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-
-	/** 
-	 * A map function that assigns an initial rank to all pages. 
-	 */
-	public static final class RankAssigner implements MapFunction<Long, Tuple2<Long, Double>> {
-		Tuple2<Long, Double> outPageWithRank;
-		
-		public RankAssigner(double rank) {
-			this.outPageWithRank = new Tuple2<Long, Double>(-1l, rank);
-		}
-		
-		@Override
-		public Tuple2<Long, Double> map(Long page) {
-			outPageWithRank.f0 = page;
-			return outPageWithRank;
-		}
-	}
-	
-	/**
-	 * A reduce function that takes a sequence of edges and builds the adjacency list for the vertex where the edges
-	 * originate. Run as a pre-processing step.
-	 */
-	@ConstantFields("0")
-	public static final class BuildOutgoingEdgeList implements GroupReduceFunction<Tuple2<Long, Long>, Tuple2<Long, Long[]>> {
-		
-		private final ArrayList<Long> neighbors = new ArrayList<Long>();
-		
-		@Override
-		public void reduce(Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long[]>> out) {
-			neighbors.clear();
-			Long id = 0L;
-			
-			for (Tuple2<Long, Long> n : values) {
-				id = n.f0;
-				neighbors.add(n.f1);
-			}
-			out.collect(new Tuple2<Long, Long[]>(id, neighbors.toArray(new Long[neighbors.size()])));
-		}
-	}
-	
-	/**
-	 * Join function that distributes a fraction of a vertex's rank to all neighbors.
-	 */
-	public static final class JoinVertexWithEdgesMatch implements FlatMapFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>>, Tuple2<Long, Double>> {
-
-		@Override
-		public void flatMap(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>> value, Collector<Tuple2<Long, Double>> out){
-			Long[] neigbors = value.f1.f1;
-			double rank = value.f0.f1;
-			double rankToDistribute = rank / ((double) neigbors.length);
-				
-			for (int i = 0; i < neigbors.length; i++) {
-				out.collect(new Tuple2<Long, Double>(neigbors[i], rankToDistribute));
-			}
-		}
-	}
-	
-	/**
-	 * The function that applies the page rank dampening formula
-	 */
-	@ConstantFields("0")
-	public static final class Dampener implements MapFunction<Tuple2<Long,Double>, Tuple2<Long,Double>> {
-
-		private final double dampening;
-		private final double randomJump;
-		
-		public Dampener(double dampening, double numVertices) {
-			this.dampening = dampening;
-			this.randomJump = (1 - dampening) / numVertices;
-		}
-
-		@Override
-		public Tuple2<Long, Double> map(Tuple2<Long, Double> value) {
-			value.f1 = (value.f1 * dampening) + randomJump;
-			return value;
-		}
-	}
-	
-	/**
-	 * Filter that filters vertices where the rank difference is below a threshold.
-	 */
-	public static final class EpsilonFilter implements FilterFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>>> {
-
-		@Override
-		public boolean filter(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>> value) {
-			return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
-		}
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String pagesInputPath = null;
-	private static String linksInputPath = null;
-	private static String outputPath = null;
-	private static long numPages = 0;
-	private static int maxIterations = 10;
-	
-	private static boolean parseParameters(String[] args) {
-		
-		if(args.length > 0) {
-			if(args.length == 5) {
-				fileOutput = true;
-				pagesInputPath = args[0];
-				linksInputPath = args[1];
-				outputPath = args[2];
-				numPages = Integer.parseInt(args[3]);
-				maxIterations = Integer.parseInt(args[4]);
-			} else {
-				System.err.println("Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing PageRank Basic example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
-			
-			numPages = PageRankData.getNumberOfPages();
-		}
-		return true;
-	}
-	
-	private static DataSet<Long> getPagesDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return env
-						.readCsvFile(pagesInputPath)
-							.fieldDelimiter(' ')
-							.lineDelimiter("\n")
-							.types(Long.class)
-						.map(new MapFunction<Tuple1<Long>, Long>() {
-							@Override
-							public Long map(Tuple1<Long> v) { return v.f0; }
-						});
-		} else {
-			return PageRankData.getDefaultPagesDataSet(env);
-		}
-	}
-	
-	private static DataSet<Tuple2<Long, Long>> getLinksDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return env.readCsvFile(linksInputPath)
-						.fieldDelimiter(' ')
-						.lineDelimiter("\n")
-						.types(Long.class, Long.class);
-		} else {
-			return PageRankData.getDefaultEdgeDataSet(env);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
deleted file mode 100644
index 0745d73..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.graph;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.example.java.graph.util.ConnectedComponentsData;
-import org.apache.flink.util.Collector;
-
-import java.util.HashSet;
-import java.util.Set;
-
-@SuppressWarnings("serial")
-public class TransitiveClosureNaive implements ProgramDescription {
-
-
-	public static void main (String... args) throws Exception{
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up execution environment
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env);
-
-		IterativeDataSet<Tuple2<Long,Long>> paths = edges.iterate(maxIterations);
-
-		DataSet<Tuple2<Long,Long>> nextPaths = paths
-				.join(edges)
-				.where(1)
-				.equalTo(0)
-				.with(new JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>>() {
-					@Override
-					/**
-						left: Path (z,x) - x is reachable by z
-						right: Edge (x,y) - edge x-->y exists
-						out: Path (z,y) - y is reachable by z
-					 */
-					public Tuple2<Long, Long> join(Tuple2<Long, Long> left, Tuple2<Long, Long> right) throws Exception {
-						return new Tuple2<Long, Long>(
-								new Long(left.f0),
-								new Long(right.f1));
-					}
-				})
-				.union(paths)
-				.groupBy(0, 1)
-				.reduceGroup(new GroupReduceFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>() {
-					@Override
-					public void reduce(Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) throws Exception {
-						out.collect(values.iterator().next());
-					}
-				});
-
-		DataSet<Tuple2<Long,Long>> newPaths = paths
-				.coGroup(nextPaths)
-				.where(0).equalTo(0)
-				.with(new CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>>() {
-					Set prevSet = new HashSet<Tuple2<Long,Long>>();
-					@Override
-					public void coGroup(Iterable<Tuple2<Long, Long>> prevPaths, Iterable<Tuple2<Long, Long>> nextPaths, Collector<Tuple2<Long, Long>> out) throws Exception {
-						for (Tuple2<Long,Long> prev : prevPaths) {
-							prevSet.add(prev);
-						}
-						for (Tuple2<Long,Long> next: nextPaths) {
-							if (!prevSet.contains(next)) {
-								out.collect(next);
-							}
-						}
-					}
-				});
-
-		DataSet<Tuple2<Long, Long>> transitiveClosure = paths.closeWith(nextPaths);
-
-
-		// emit result
-		if (fileOutput) {
-			transitiveClosure.writeAsCsv(outputPath, "\n", " ");
-		} else {
-			transitiveClosure.print();
-		}
-
-		// execute program
-		env.execute("Transitive Closure Example");
-
-	}
-
-	@Override
-	public String getDescription() {
-		return "Parameters: <edges-path> <result-path> <max-number-of-iterations>";
-	}
-
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String edgesPath = null;
-	private static String outputPath = null;
-	private static int maxIterations = 10;
-
-	private static boolean parseParameters(String[] programArguments) {
-
-		if (programArguments.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (programArguments.length == 3) {
-				edgesPath = programArguments[0];
-				outputPath = programArguments[1];
-				maxIterations = Integer.parseInt(programArguments[2]);
-			} else {
-				System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of iterations>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing TransitiveClosure example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: TransitiveClosure <edges path> <result path> <max number of iterations>");
-		}
-		return true;
-	}
-
-
-	private static DataSet<Tuple2<Long, Long>> getEdgeDataSet(ExecutionEnvironment env) {
-
-		if(fileOutput) {
-			return env.readCsvFile(edgesPath).fieldDelimiter(' ').types(Long.class, Long.class);
-		} else {
-			return ConnectedComponentsData.getDefaultEdgeDataSet(env);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
deleted file mode 100644
index aa5bbd6..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.graph.util;
-
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * Provides the default data sets used for the Connected Components example program.
- * The default data sets are used, if no parameters are given to the program.
- *
- */
-public class ConnectedComponentsData {
-	
-	public static final Object[][] VERTICES  = new Object[][] {
-		new Object[]{1L}, new Object[]{2L}, new Object[]{3L}, new Object[]{4L}, 
-		new Object[]{5L},new Object[]{6L}, new Object[]{7L}, new Object[]{8L}, 
-		new Object[]{9L}, new Object[]{10L}, new Object[]{11L}, new Object[]{12L}, 
-		new Object[]{13L}, new Object[]{14L}, new Object[]{15L}, new Object[]{16L}
-	};
-
-	public static DataSet<Long> getDefaultVertexDataSet(ExecutionEnvironment env) {
-		List<Long> verticesList = new LinkedList<Long>();
-		for (Object[] vertex : VERTICES) {
-			verticesList.add((Long) vertex[0]);
-		}
-		return env.fromCollection(verticesList);
-	}
-	
-	public static final Object[][] EDGES = new Object[][] {
-		new Object[]{1L, 2L},
-		new Object[]{2L, 3L},
-		new Object[]{2L, 4L},
-		new Object[]{3L, 5L},
-		new Object[]{6L, 7L},
-		new Object[]{8L, 9L},
-		new Object[]{8L, 10L},
-		new Object[]{5L, 11L},
-		new Object[]{11L, 12L},
-		new Object[]{10L, 13L},
-		new Object[]{9L, 14L},
-		new Object[]{13L, 14L},
-		new Object[]{1L, 15L},
-		new Object[]{16L, 1L}
-	};
-	
-	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-		
-		List<Tuple2<Long, Long>> edgeList = new LinkedList<Tuple2<Long, Long>>();
-		for (Object[] edge : EDGES) {
-			edgeList.add(new Tuple2<Long, Long>((Long) edge[0], (Long) edge[1]));
-		}
-		return env.fromCollection(edgeList);
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
deleted file mode 100644
index 02c0531..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.graph.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Edge;
-
-/**
- * Provides the default data sets used for the Triangle Enumeration example programs.
- * The default data sets are used, if no parameters are given to the program.
- *
- */
-public class EnumTrianglesData {
-
-	public static final Object[][] EDGES = {
-		{1, 2},
-		{1, 3},
-		{1 ,4},
-		{1, 5},
-		{2, 3},
-		{2, 5},
-		{3, 4},
-		{3, 7},
-		{3, 8},
-		{5, 6},
-		{7, 8}
-	};
-	
-	public static DataSet<Edge> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-		
-		List<Edge> edges = new ArrayList<Edge>();
-		for(Object[] e : EDGES) {
-			edges.add(new Edge((Integer)e[0], (Integer)e[1]));
-		}
-		
-		return env.fromCollection(edges);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesDataTypes.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesDataTypes.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesDataTypes.java
deleted file mode 100644
index 95c0c85..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesDataTypes.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.graph.util;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-
-public class EnumTrianglesDataTypes {
-
-	public static class Edge extends Tuple2<Integer, Integer> {
-		private static final long serialVersionUID = 1L;
-		
-		public static final int V1 = 0;
-		public static final int V2 = 1;
-		
-		public Edge() {}
-		
-		public Edge(final Integer v1, final Integer v2) {
-			this.setFirstVertex(v1);
-			this.setSecondVertex(v2);
-		}
-		
-		public Integer getFirstVertex() { return this.getField(V1); }
-		
-		public Integer getSecondVertex() { return this.getField(V2); }
-		
-		public void setFirstVertex(final Integer vertex1) { this.setField(vertex1, V1); }
-		
-		public void setSecondVertex(final Integer vertex2) { this.setField(vertex2, V2); }
-		
-		public void copyVerticesFromTuple2(Tuple2<Integer, Integer> t) {
-			this.setFirstVertex(t.f0);
-			this.setSecondVertex(t.f1);
-		}
-		
-		public void copyVerticesFromEdgeWithDegrees(EdgeWithDegrees ewd) {
-			this.setFirstVertex(ewd.getFirstVertex());
-			this.setSecondVertex(ewd.getSecondVertex());
-		}
-		
-		public void flipVertices() {
-			Integer tmp = this.getFirstVertex();
-			this.setFirstVertex(this.getSecondVertex());
-			this.setSecondVertex(tmp);
-		}
-	}
-	
-	public static class Triad extends Tuple3<Integer, Integer, Integer> {
-		private static final long serialVersionUID = 1L;
-		
-		public static final int V1 = 0;
-		public static final int V2 = 1;
-		public static final int V3 = 2;
-		
-		public Triad() {}
-		
-		public void setFirstVertex(final Integer vertex1) { this.setField(vertex1, V1); }
-		
-		public void setSecondVertex(final Integer vertex2) { this.setField(vertex2, V2); }
-		
-		public void setThirdVertex(final Integer vertex3) { this.setField(vertex3, V3); }
-	}
-	
-	public static class EdgeWithDegrees extends Tuple4<Integer, Integer, Integer, Integer> {
-		private static final long serialVersionUID = 1L;
-
-		public static final int V1 = 0;
-		public static final int V2 = 1;
-		public static final int D1 = 2;
-		public static final int D2 = 3;
-		
-		public EdgeWithDegrees() { }
-			
-		public Integer getFirstVertex() { return this.getField(V1); }
-		
-		public Integer getSecondVertex() { return this.getField(V2); }
-		
-		public Integer getFirstDegree() { return this.getField(D1); }
-		
-		public Integer getSecondDegree() { return this.getField(D2); }
-		
-		public void setFirstVertex(final Integer vertex1) { this.setField(vertex1, V1); }
-		
-		public void setSecondVertex(final Integer vertex2) { this.setField(vertex2, V2); }
-		
-		public void setFirstDegree(final Integer degree1) { this.setField(degree1, D1); }
-		
-		public void setSecondDegree(final Integer degree2) { this.setField(degree2, D2); }
-		
-		public void copyFrom(final EdgeWithDegrees edge) {
-			this.setFirstVertex(edge.getFirstVertex());
-			this.setSecondVertex(edge.getSecondVertex());
-			this.setFirstDegree(edge.getFirstDegree());
-			this.setSecondDegree(edge.getSecondDegree());
-		}
-	}
-	
-	
-}


[56/60] git commit: Turn Documentation into standalone website, add Overview Page

Posted by al...@apache.org.
Turn Documentation into standalone website, add Overview Page

This can now be built standalone and then copied into the correct docs
folder of the website SVN.

The index page now has a short overview and a table of contents.


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

Branch: refs/heads/master
Commit: b8a8780f4e0b8b9727a862df318ac4ed522c698c
Parents: 15060ef
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 19 14:47:43 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 13:42:57 2014 +0200

----------------------------------------------------------------------
 .gitignore                          |    2 +
 docs/_config.yml                    |    9 +-
 docs/_includes/footer.md            |   14 +
 docs/_includes/navbar.html          |   27 +
 docs/_includes/sidenav.html         |   52 +
 docs/_layouts/default.html          |   73 +
 docs/_layouts/docs.html             |   96 -
 docs/_plugins/build_apidoc.rb       |   51 +
 docs/css/bootstrap-lumen-custom.css | 6189 ++++++++++++++++++++++++++++++
 docs/css/bootstrap.css              | 5785 ++++++++++++++++++++++++++++
 docs/css/custom.css                 |    9 +
 docs/dataset_transformations.md     |  608 ++-
 docs/hadoop_compatability.md        |    5 -
 docs/hadoop_compatibility.md        |    5 +
 docs/img/apache-incubator-logo.png  |  Bin 0 -> 4234 bytes
 docs/index.md                       |   22 +-
 docs/js/bootstrap.min.js            |    6 +
 docs/programming_guide.md           |    4 +-
 18 files changed, 12731 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 1bca0cf..4e258aa 100644
--- a/.gitignore
+++ b/.gitignore
@@ -15,3 +15,5 @@ tmp
 *.swp
 *.jar
 .DS_Store
+_site
+docs/api

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 6652886..8a5690e 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -6,11 +6,16 @@
 #------------------------------------------------------------------------------
 
 FLINK_VERSION_STABLE: 0.7-incubating # this variable can point to a SNAPSHOT version in the git source.
-FLINK_VERSION_HADOOP_2_STABLE: 0.7-hadoop2-incubating
 FLINK_VERSION_SHORT: 0.7
+FLINK_VERSION_HADOOP_2_STABLE: 0.7-hadoop2-incubating
+FLINK_SCALA_VERSION: 2.10.4
+FLINK_SCALA_VERSION_SHORT: 2.10
 FLINK_ISSUES_URL: https://issues.apache.org/jira/browse/FLINK
 FLINK_GITHUB_URL:  https://github.com/apache/incubator-flink
 
+FLINK_WEBSITE_URL: http://flink.incubator.apache.org
+FLINK_DOWNLOAD_URL: http://flink.incubator.apache.org/downloads.html
+
 FLINK_DOWNLOAD_URL_HADOOP_1_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.7-incubating-bin-hadoop1.tgz
 FLINK_DOWNLOAD_URL_HADOOP_2_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.7-incubating-bin-hadoop2.tgz
 FLINK_DOWNLOAD_URL_YARN_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.7-incubating-bin-hadoop2-yarn.tgz
@@ -27,7 +32,7 @@ defaults:
     scope:
       path: ""
     values:
-      layout: docs
+      layout: default
 
 highlighter: pygments
 markdown: KramdownPygments

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_includes/footer.md
----------------------------------------------------------------------
diff --git a/docs/_includes/footer.md b/docs/_includes/footer.md
new file mode 100644
index 0000000..b96adde
--- /dev/null
+++ b/docs/_includes/footer.md
@@ -0,0 +1,14 @@
+<hr class="divider">
+
+<small>Apache Flink is an effort undergoing incubation at The Apache Software
+Foundation (ASF), sponsored by the Apache Incubator PMC. Incubation is
+required of all newly accepted projects until a further review indicates that
+the infrastructure, communications, and decision making process have
+stabilized in a manner consistent with other successful ASF projects. While
+incubation status is not necessarily a reflection of the completeness or
+stability of the code, it does indicate that the project has yet to be fully
+endorsed by the ASF.</small>
+
+<a href="http://incubator.apache.org/">![Incubator Logo]({{ site.baseurl }}/img/apache-incubator-logo.png)</a>
+
+<p class="text-center"><a href="{{ site.baseurl }}/privacy-policy.html">Privacy Policy<a></p>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_includes/navbar.html
----------------------------------------------------------------------
diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html
new file mode 100644
index 0000000..5c7b8ac
--- /dev/null
+++ b/docs/_includes/navbar.html
@@ -0,0 +1,27 @@
+<nav class="navbar navbar-default navbar-fixed-top" role="navigation">
+  <div class="container">
+    <div class="navbar-header">
+      <button type="button" class="navbar-toggle" data-toggle="collapse" data-target=".navbar-collapse">
+        <span class="sr-only">Toggle navigation</span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+      </button>
+      <a class="navbar-brand" href="http://flink.incubator.apache.org">Apache Flink</a>
+    </div>
+
+    <div class="collapse navbar-collapse" id="navbar-collapse-1">
+      <ul class="nav navbar-nav">
+
+        <li>
+          <a href="index.html" class="{% if page.url contains 'index.html' %}active{% endif %}">Documentation</a>
+        </li>
+
+        <li>
+          <a href="api/java/index.html">Javadoc</a>
+        </li>
+
+      </ul>
+    </div>
+  </div>
+</nav>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_includes/sidenav.html
----------------------------------------------------------------------
diff --git a/docs/_includes/sidenav.html b/docs/_includes/sidenav.html
new file mode 100644
index 0000000..783c5c4
--- /dev/null
+++ b/docs/_includes/sidenav.html
@@ -0,0 +1,52 @@
+<ul>
+  <li><a href="faq.html">FAQ</a></li>
+  <li>Quickstart
+    <ul>
+      <li><a href="setup_quickstart.html">Setup</a></li>
+      <li><a href="run_example_quickstart.html">Run Example</a></li>
+      <li><a href="java_api_quickstart.html">Java API</a></li>
+      <li><a href="scala_api_quickstart.html">Scala API</a></li>
+    </ul>
+  </li>
+
+  <li>Setup &amp; Configuration
+    <ul>
+      <li><a href="local_setup.html">Local Setup</a></li>
+      <li><a href="building.html">Build Flink</a></li>
+      <li><a href="cluster_setup.html">Cluster Setup</a></li>
+      <li><a href="yarn_setup.html">YARN Setup</a></li>
+      <li><a href="config.html">Configuration</a></li>
+    </ul>
+  </li>
+
+  <li>Programming Guides
+    <ul>
+      <li><a href="programming_guide.html">Programming Guide</a></li>
+      <li><a href="dataset_transformations.html">DataSet Transformations</a></li>
+      <li><a href="iterations.html">Iterations</a></li>
+      <li><a href="spargel_guide.html">Spargel Graph API</a></li>
+    </ul>
+  </li>
+
+  <li>Examples
+    <ul>
+      <li><a href="examples.html">Bundled Examples</a></li>
+      <li><a href="example_connectors.html">Connecting to other systems</a></li>
+    </ul>
+  </li>
+
+  <li>Execution
+    <ul>
+      <li><a href="local_execution.html">Local/Debugging</a></li>
+      <li><a href="cluster_execution.html">Cluster</a></li>
+      <li><a href="cli.html">Command-Line Interface</a></li>
+      <li><a href="web_client.html">Web Interface</a></li>
+    </ul>
+  </li>
+
+  <li>Internals
+    <ul>
+      <li><a href="internal_overview.html">Overview</a></li>
+    </ul>
+  </li>
+</ul>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_layouts/default.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html
new file mode 100644
index 0000000..f33a424
--- /dev/null
+++ b/docs/_layouts/default.html
@@ -0,0 +1,73 @@
+<!DOCTYPE html>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Apache Flink (incubating): {{ page.title }}</title>
+    <link rel="stylesheet" href="{{ site.baseurl }}/css/bootstrap.css">
+    <link rel="stylesheet" href="{{ site.baseurl }}/css/bootstrap-lumen-custom.css">
+    <link rel="stylesheet" href="{{ site.baseurl }}/css/syntax.css">
+    <link rel="/css/custom.css">
+    <link rel="css/codetabs.css">
+    <link href="//maxcdn.bootstrapcdn.com/font-awesome/4.1.0/css/font-awesome.min.css" rel="stylesheet">
+    <script src="https://ajax.googleapis.com/ajax/libs/jquery/1.11.0/jquery.min.js"></script>
+    <script src="/js/bootstrap.min.js"></script>
+    <script src="js/codetabs.js"></script>
+  </head>
+  <body>
+
+    {% include navbar.html %}
+
+    <div style="padding-top:70px" class="container">
+
+      <div class="row">
+        <div class="col-md-3">
+          {% include sidenav.html %}
+        </div>  
+        <div class="col-md-9">
+          <h1>{{ page.title }}</h1>
+
+          {{ content }}
+
+          <div style="padding-top:30px" id="disqus_thread"></div>
+      
+            <script type="text/javascript">
+                /* * * CONFIGURATION VARIABLES: EDIT BEFORE PASTING INTO YOUR WEBPAGE * * */
+                var disqus_shortname = 'stratosphere-eu'; // required: replace example with your forum shortname
+
+                /* * * DON'T EDIT BELOW THIS LINE * * */
+                (function() {
+                    var dsq = document.createElement('script'); dsq.type = 'text/javascript'; dsq.async = true;
+                    dsq.src = '//' + disqus_shortname + '.disqus.com/embed.js';
+                    (document.getElementsByTagName('head')[0] || document.getElementsByTagName('body')[0]).appendChild(dsq);
+                })();
+            </script>
+            <noscript>Please enable JavaScript to view the <a href="http://disqus.com/?ref_noscript">comments powered by Disqus.</a></noscript>
+            <a href="http://disqus.com" class="dsq-brlink">comments powered by <span class="logo-disqus">Disqus</span></a>
+
+          </div>
+        </div>
+
+        <div class="footer">
+          {% capture footer %}{% include footer.md %}{% endcapture %}
+          {{ footer | markdownify }}
+        </div>
+      </div>
+    </div>
+
+    
+
+    <script>
+      (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
+      (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
+      m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
+      })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
+
+      ga('create', 'UA-52545728-1', 'auto');
+      ga('send', 'pageview');
+
+    </script>
+
+  </body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/docs.html b/docs/_layouts/docs.html
deleted file mode 100644
index f85e75a..0000000
--- a/docs/_layouts/docs.html
+++ /dev/null
@@ -1,96 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-  <head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <title>Apache Flink {{ site.FLINK_VERSION_STABLE }} Documentation: {{ page.title }}</title>
-
-    <link rel="stylesheet" href="//netdna.bootstrapcdn.com/bootstrap/3.1.1/css/bootstrap.min.css">
-    <link rel="stylesheet" href="css/syntax.css">
-    <link rel="stylesheet" href="css/codetabs.css">
-
-    <!-- HTML5 Shim and Respond.js IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-      <script src="https://oss.maxcdn.com/libs/html5shiv/3.7.0/html5shiv.js"></script>
-      <script src="https://oss.maxcdn.com/libs/respond.js/1.4.2/respond.min.js"></script>
-    <![endif]-->
-  </head>
-  <body>
-    <div class="container">
-        <div class="row">
-            <h1>Apache Flink {{ site.FLINK_VERSION_STABLE }} Documentation</h1>
-        </div>
-        <div class="row">
-            <div class="col-md-3">
-                <ul>
-                    <li>Quickstart
-                        <ul>
-                            <li><a href="setup_quickstart.html">Install</a></li>
-                            <li><a href="run_example_quickstart.html">Run Example</a></li>
-                            <li><a href="java_api_quickstart.html">Java API</a></li>
-                            <li><a href="scala_api_quickstart.html">Scala API</a></li>
-                            <li><a href="faq.html">FAQ</a></li>
-                        </ul>
-                    </li>
-
-                    <li>Setup &amp; Configuration
-                        <ul>
-                            <li><a href="building.html">Build Instructions</a></li>
-                            <li><a href="local_setup.html">Local Setup</a></li>
-                            <li><a href="cluster_setup.html">Cluster Setup</a></li>
-                            <li><a href="yarn_setup.html">YARN Setup</a></li>
-                            <li><a href="config.html">Configuration</a></li>
-                        </ul>
-                    </li>
-
-                    <li>Programming Guides
-                        <ul>
-                            <li><a href="programming_guide.html">Flink Programming Guide</a></li>
-                            <li><a href="dataset_transformations.html">DataSet Transformations</a></li>
-                            <li><a href="hadoop_compatability.html">Hadoop Compatability</a></li>
-                            <li><a href="iterations.html">Iterations</a></li>
-                            <li><a href="spargel_guide.html">Spargel Graph API</a></li>
-                        </ul>
-                    </li>
-
-                    <li>Examples
-                        <ul>
-                            <li><a href="examples.html">Bundled Examples</a></li>
-                            <li><a href="example_connectors.html">Connecting to other systems</a></li>
-                        </ul>
-                    </li>
-
-                    <li>Execution
-                        <ul>
-                            <li><a href="local_execution.html">Local/Debugging</a></li>
-                            <li><a href="cluster_execution.html">Cluster</a></li>
-                            <li><a href="cli.html">Command-Line Interface</a></li>
-                            <li><a href="web_client.html">Web Interface</a></li>
-                        </ul>
-                    </li>
-
-                    <li>Internals
-                        <ul>
-                            <li><a href="internal_overview.html">Overview</a></li>
-                            <li><a href="internal_general_arch.html">General Architecture</a></li>
-                            <li><a href="internal_add_operator.html">How-to: Adding a new Operator</a></li>
-                            <li><a href="internal_logging.html">How-to: Logging</a></li>
-                        </ul>
-                    </li>
-                </ul>
-            </div>
-            <div class="col-md-9">
-                <h1>{{ page.title }}</h1>
-
-                {{ content }}
-            </div>
-        </div>
-    </div>
-
-    <script src="https://ajax.googleapis.com/ajax/libs/jquery/1.11.0/jquery.min.js"></script>
-    <script src="//netdna.bootstrapcdn.com/bootstrap/3.1.1/js/bootstrap.min.js"></script>
-    <script src="js/codetabs.js"></script>
-  </body>
-</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/_plugins/build_apidoc.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/build_apidoc.rb b/docs/_plugins/build_apidoc.rb
new file mode 100644
index 0000000..8caa971
--- /dev/null
+++ b/docs/_plugins/build_apidoc.rb
@@ -0,0 +1,51 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+require 'fileutils'
+require 'rubygems'
+include FileUtils
+
+if ENV['BUILD_API'] == '1' then
+  # Build Javadoc and Scaladoc
+
+  cd("..")
+
+  java8_doc_fix = ""
+
+  java_ver = Gem::Version.new(`java -version 2>&1 | awk -F '"' '/version/ {print $2}'`.gsub('_', '.'))
+
+  if java_ver > Gem::Version.new("1.8") then
+    puts "Detected Java8, adding -Xdoclint:none"
+    java8_doc_fix = '-DadditionalJOption="-Xdoclint:none"'
+  end
+
+  puts "Running mvn clean install -DskipTests"
+  puts `mvn clean install -DskipTests`
+
+  puts "Generating Javadoc"
+  puts `mvn javadoc:aggregate -Pdocs-and-source $JAVA8JAVADOCFIX -Dmaven.javadoc.failOnError=false -Dquiet=true -Dheader="<a href=\"/\" target=\"_top\"><h1>Back to Flink Documentation</h1></a>"`
+  
+  cd("docs")
+
+  mkdir_p "api"
+
+  source = "../target/site/apidocs"
+  dest = "api/java/"
+
+  puts "cp -r " + source + "/. " + dest
+  cp_r(source + "/.", dest)
+
+end


[23/60] Renamed java examples package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/RelationalQuery.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/RelationalQuery.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/RelationalQuery.java
new file mode 100644
index 0000000..ba06ec4
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/RelationalQuery.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.relational;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * This program implements the following relational query on the TPC-H data set.
+ * 
+ * <p>
+ * <code><pre>
+ * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+ *   FROM orders, lineitem
+ *   WHERE l_orderkey = o_orderkey
+ *     AND o_orderstatus = "X"
+ *     AND YEAR(o_orderdate) > Y
+ *     AND o_orderpriority LIKE "Z%"
+ *   GROUP BY l_orderkey, o_shippriority;
+ * </pre></code>
+ *        
+ * <p>
+ * Input files are plain text CSV files using the pipe character ('|') as field separator 
+ * as generated by the TPC-H data generator which is available at <a href="http://www.tpc.org/tpch/">http://www.tpc.org/tpch/</a>.
+ * 
+ * <p>
+ * Usage: <code>RelationalQuery &lt;orders-csv path&gt; &lt;lineitem-csv path&gt; &lt;result path&gt;</code><br>
+ *  
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> tuple data types
+ * <li> inline-defined functions
+ * <li> projection and join projection
+ * <li> build-in aggregation functions
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class RelationalQuery {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	private static String STATUS_FILTER = "F";
+	private static int YEAR_FILTER = 1993;
+	private static String OPRIO_FILTER = "5";
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
+		DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
+
+		// get lineitem data set: (orderkey, extendedprice)
+		DataSet<Tuple2<Integer, Double>> lineitems = getLineitemDataSet(env);
+
+		// orders filtered by year: (orderkey, custkey)
+		DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
+				// filter orders
+				orders.filter(
+								new FilterFunction<Tuple5<Integer, String, String, String, Integer>>() {
+									@Override
+									public boolean filter(Tuple5<Integer, String, String, String, Integer> t) {
+										// status filter
+										if(!t.f1.equals(STATUS_FILTER)) {
+											return false;
+										// year filter
+										} else if(Integer.parseInt(t.f2.substring(0, 4)) <= YEAR_FILTER) {
+											return false;
+										// order priority filter
+										} else if(!t.f3.startsWith(OPRIO_FILTER)) {
+											return false;
+										}
+										return true;
+									}
+								})
+				// project fields out that are no longer required
+				.project(0,4).types(Integer.class, Integer.class);
+
+		// join orders with lineitems: (orderkey, shippriority, extendedprice)
+		DataSet<Tuple3<Integer, Integer, Double>> lineitemsOfOrders = 
+				ordersFilteredByYear.joinWithHuge(lineitems)
+									.where(0).equalTo(0)
+									.projectFirst(0,1).projectSecond(1)
+									.types(Integer.class, Integer.class, Double.class);
+
+		// extendedprice sums: (orderkey, shippriority, sum(extendedprice))
+		DataSet<Tuple3<Integer, Integer, Double>> priceSums =
+				// group by order and sum extendedprice
+				lineitemsOfOrders.groupBy(0,1).aggregate(Aggregations.SUM, 2);
+
+		// emit result
+		priceSums.writeAsCsv(outputPath);
+		
+		// execute program
+		env.execute("Relational Query Example");
+		
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static String ordersPath;
+	private static String lineitemPath;
+	private static String outputPath;
+	
+	private static boolean parseParameters(String[] programArguments) {
+		
+		if(programArguments.length > 0) {
+			if(programArguments.length == 3) {
+				ordersPath = programArguments[0];
+				lineitemPath = programArguments[1];
+				outputPath = programArguments[2];
+			} else {
+				System.err.println("Usage: RelationalQuery <orders-csv path> <lineitem-csv path> <result path>");
+				return false;
+			}
+		} else {
+			System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
+								"  Due to legal restrictions, we can not ship generated data.\n" +
+								"  You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + 
+								"  Usage: RelationalQuery <orders-csv path> <lineitem-csv path> <result path>");
+			return false;
+		}
+		return true;
+	}
+	
+	private static DataSet<Tuple5<Integer, String, String, String, Integer>> getOrdersDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(ordersPath)
+					.fieldDelimiter('|')
+					.includeFields("101011010")
+					.types(Integer.class, String.class, String.class, String.class, Integer.class);
+	}
+
+	private static DataSet<Tuple2<Integer, Double>> getLineitemDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(lineitemPath)
+					.fieldDelimiter('|')
+					.includeFields("1000010000000000")
+					.types(Integer.class, Double.class);
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
new file mode 100644
index 0000000..a8978fd
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
@@ -0,0 +1,247 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.relational;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * This program implements a modified version of the TPC-H query 10.
+ * The original query can be found at
+ * <a href="http://www.tpc.org/tpch/spec/tpch2.16.0.pdf">http://www.tpc.org/tpch/spec/tpch2.16.0.pdf</a> (page 45).
+ * 
+ * <p>
+ * This program implements the following SQL equivalent:
+ * 
+ * <p>
+ * <code><pre>
+ * SELECT 
+ *        c_custkey,
+ *        c_name, 
+ *        c_address,
+ *        n_name, 
+ *        c_acctbal
+ *        SUM(l_extendedprice * (1 - l_discount)) AS revenue,  
+ * FROM   
+ *        customer, 
+ *        orders, 
+ *        lineitem, 
+ *        nation 
+ * WHERE 
+ *        c_custkey = o_custkey 
+ *        AND l_orderkey = o_orderkey 
+ *        AND YEAR(o_orderdate) > '1990' 
+ *        AND l_returnflag = 'R' 
+ *        AND c_nationkey = n_nationkey 
+ * GROUP BY 
+ *        c_custkey, 
+ *        c_name, 
+ *        c_acctbal, 
+ *        n_name, 
+ *        c_address
+ * </pre></code>
+ *        
+ * <p>
+ * Compared to the original TPC-H query this version does not print 
+ * c_phone and c_comment, only filters by years greater than 1990 instead of
+ * a period of 3 months, and does not sort the result by revenue.
+ * 
+ * <p>
+ * Input files are plain text CSV files using the pipe character ('|') as field separator 
+ * as generated by the TPC-H data generator which is available at <a href="http://www.tpc.org/tpch/">http://www.tpc.org/tpch/</a>.
+ * 
+ * <p>
+ * Usage: <code>TPCHQuery10 &lt;customer-csv path&gt; &lt;orders-csv path&gt; &lt;lineitem-csv path&gt; &lt;nation-csv path&gt; &lt;result path&gt;</code><br>
+ *  
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> tuple data types
+ * <li> inline-defined functions
+ * <li> projection and join projection
+ * <li> build-in aggregation functions
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class TPCHQuery10 {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// get customer data set: (custkey, name, address, nationkey, acctbal) 
+		DataSet<Tuple5<Integer, String, String, Integer, Double>> customers = getCustomerDataSet(env);
+
+		// get orders data set: (orderkey, custkey, orderdate)
+		DataSet<Tuple3<Integer, Integer, String>> orders = getOrdersDataSet(env);
+
+		// get lineitem data set: (orderkey, extendedprice, discount, returnflag)
+		DataSet<Tuple4<Integer, Double, Double, String>> lineitems = getLineitemDataSet(env);
+
+		// get nation data set: (nationkey, name)
+		DataSet<Tuple2<Integer, String>> nations = getNationsDataSet(env);
+
+		// orders filtered by year: (orderkey, custkey)
+		DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
+				// filter by year
+				orders.filter(
+								new FilterFunction<Tuple3<Integer,Integer, String>>() {
+									@Override
+									public boolean filter(Tuple3<Integer, Integer, String> t) {
+										int year = Integer.parseInt(t.f2.substring(0, 4));
+										return year > 1990;
+									}
+								})
+				// project fields out that are no longer required
+				.project(0,1).types(Integer.class, Integer.class);
+
+		// lineitems filtered by flag: (orderkey, extendedprice, discount)
+		DataSet<Tuple3<Integer, Double, Double>> lineitemsFilteredByFlag = 
+				// filter by flag
+				lineitems.filter(new FilterFunction<Tuple4<Integer, Double, Double, String>>() {
+										@Override
+										public boolean filter(Tuple4<Integer, Double, Double, String> t)
+												throws Exception {
+											return t.f3.equals("R");
+										}
+								})
+				// project fields out that are no longer required
+				.project(0,1,2).types(Integer.class, Double.class, Double.class);
+
+		// join orders with lineitems: (custkey, extendedprice, discount)
+		DataSet<Tuple3<Integer, Double, Double>> lineitemsOfCustomerKey = 
+				ordersFilteredByYear.joinWithHuge(lineitemsFilteredByFlag)
+									.where(0).equalTo(0)
+									.projectFirst(1).projectSecond(1,2)
+									.types(Integer.class, Double.class, Double.class);
+
+		// aggregate for revenue: (custkey, revenue)
+		DataSet<Tuple2<Integer, Double>> revenueOfCustomerKey = lineitemsOfCustomerKey
+				// calculate the revenue for each item
+				.map(new MapFunction<Tuple3<Integer, Double, Double>, Tuple2<Integer, Double>>() {
+							@Override
+							public Tuple2<Integer, Double> map(Tuple3<Integer, Double, Double> t) {
+								// revenue per item = l_extendedprice * (1 - l_discount)
+								return new Tuple2<Integer, Double>(t.f0, t.f1 * (1 - t.f2));
+							}
+					})
+				// aggregate the revenues per item to revenue per customer
+				.groupBy(0).aggregate(Aggregations.SUM, 1);
+
+		// join customer with nation (custkey, name, address, nationname, acctbal)
+		DataSet<Tuple5<Integer, String, String, String, Double>> customerWithNation = customers
+						.joinWithTiny(nations)
+						.where(3).equalTo(0)
+						.projectFirst(0,1,2).projectSecond(1).projectFirst(4)
+						.types(Integer.class, String.class, String.class, String.class, Double.class);
+
+		// join customer (with nation) with revenue (custkey, name, address, nationname, acctbal, revenue)
+		DataSet<Tuple6<Integer, String, String, String, Double, Double>> customerWithRevenue = 
+				customerWithNation.join(revenueOfCustomerKey)
+				.where(0).equalTo(0)
+				.projectFirst(0,1,2,3,4).projectSecond(1)
+				.types(Integer.class, String.class, String.class, String.class, Double.class, Double.class);
+
+		// emit result
+		customerWithRevenue.writeAsCsv(outputPath);
+		
+		// execute program
+		env.execute("TPCH Query 10 Example");
+		
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static String customerPath;
+	private static String ordersPath;
+	private static String lineitemPath;
+	private static String nationPath;
+	private static String outputPath;
+	
+	private static boolean parseParameters(String[] programArguments) {
+		
+		if(programArguments.length > 0) {
+			if(programArguments.length == 5) {
+				customerPath = programArguments[0];
+				ordersPath = programArguments[1];
+				lineitemPath = programArguments[2];
+				nationPath = programArguments[3];
+				outputPath = programArguments[4];
+			} else {
+				System.err.println("Usage: TPCHQuery10 <customer-csv path> <orders-csv path> <lineitem-csv path> <nation-csv path> <result path>");
+				return false;
+			}
+		} else {
+			System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
+								"  Due to legal restrictions, we can not ship generated data.\n" +
+								"  You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + 
+								"  Usage: TPCHQuery10 <customer-csv path> <orders-csv path> <lineitem-csv path> <nation-csv path> <result path>");
+			return false;
+		}
+		return true;
+	}
+	
+	private static DataSet<Tuple5<Integer, String, String, Integer, Double>> getCustomerDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(customerPath)
+					.fieldDelimiter('|')
+					.includeFields("11110100")
+					.types(Integer.class, String.class, String.class, Integer.class, Double.class);
+	}
+	
+	private static DataSet<Tuple3<Integer, Integer, String>> getOrdersDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(ordersPath)
+					.fieldDelimiter('|')
+					.includeFields("110010000")
+					.types(Integer.class, Integer.class, String.class);
+	}
+
+	private static DataSet<Tuple4<Integer, Double, Double, String>> getLineitemDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(lineitemPath)
+					.fieldDelimiter('|')
+					.includeFields("1000011010000000")
+					.types(Integer.class, Double.class, Double.class, String.class);
+	}
+	
+	private static DataSet<Tuple2<Integer, String>> getNationsDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(nationPath)
+					.fieldDelimiter('|')
+					.includeFields("1100")
+					.types(Integer.class, String.class);
+	}
+			
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
new file mode 100644
index 0000000..33e8c24
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
@@ -0,0 +1,298 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.relational;
+
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * This program implements a modified version of the TPC-H query 3. The
+ * example demonstrates how to assign names to fields by extending the Tuple class.
+ * The original query can be found at
+ * <a href="http://www.tpc.org/tpch/spec/tpch2.16.0.pdf">http://www.tpc.org/tpch/spec/tpch2.16.0.pdf</a> (page 29).
+ *
+ * <p>
+ * This program implements the following SQL equivalent:
+ *
+ * <p>
+ * <code><pre>
+ * SELECT 
+ *      l_orderkey, 
+ *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
+ *      o_orderdate, 
+ *      o_shippriority 
+ * FROM customer, 
+ *      orders, 
+ *      lineitem 
+ * WHERE
+ *      c_mktsegment = '[SEGMENT]' 
+ *      AND c_custkey = o_custkey
+ *      AND l_orderkey = o_orderkey
+ *      AND o_orderdate < date '[DATE]'
+ *      AND l_shipdate > date '[DATE]'
+ * GROUP BY
+ *      l_orderkey, 
+ *      o_orderdate, 
+ *      o_shippriority;
+ * </pre></code>
+ *
+ * <p>
+ * Compared to the original TPC-H query this version does not sort the result by revenue
+ * and orderdate.
+ *
+ * <p>
+ * Input files are plain text CSV files using the pipe character ('|') as field separator 
+ * as generated by the TPC-H data generator which is available at <a href="http://www.tpc.org/tpch/">http://www.tpc.org/tpch/</a>.
+ *
+ *  <p>
+ * Usage: <code>TPCHQuery3 &lt;lineitem-csv path&gt; &lt;customer-csv path&gt; &lt;orders-csv path&gt; &lt;result path&gt;</code><br>
+ *  
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> custom data type derived from tuple data types
+ * <li> inline-defined functions
+ * <li> build-in aggregation functions
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class TPCHQuery3 {
+
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataSet<Lineitem> li = getLineitemDataSet(env);
+		DataSet<Order> or = getOrdersDataSet(env);
+		DataSet<Customer> cust = getCustomerDataSet(env);
+		
+		// Filter market segment "AUTOMOBILE"
+		cust = cust.filter(
+							new FilterFunction<Customer>() {
+								@Override
+								public boolean filter(Customer value) {
+									return value.getMktsegment().equals("AUTOMOBILE");
+								}
+							});
+
+		// Filter all Orders with o_orderdate < 12.03.1995
+		or = or.filter(
+						new FilterFunction<Order>() {
+							private DateFormat format = new SimpleDateFormat("yyyy-MM-dd");
+							private Date date;
+							
+							{	
+								Calendar cal = Calendar.getInstance();
+								cal.set(1995, 3, 12);
+								date = cal.getTime(); 
+							}
+							
+							@Override
+							public boolean filter(Order value) throws ParseException {
+								Date orderDate = format.parse(value.getOrderdate());
+								return orderDate.before(date);
+							}
+						});
+		
+		// Filter all Lineitems with l_shipdate > 12.03.1995
+		li = li.filter(
+						new FilterFunction<Lineitem>() {
+							private DateFormat format = new SimpleDateFormat("yyyy-MM-dd");
+							private Date date;
+							
+							{
+								Calendar cal = Calendar.getInstance();
+								cal.set(1995, 3, 12);
+								date = cal.getTime();
+							}
+							
+							@Override
+							public boolean filter(Lineitem value) throws ParseException {
+								Date shipDate = format.parse(value.getShipdate());
+								return shipDate.after(date);
+							}
+						});
+
+		// Join customers with orders and package them into a ShippingPriorityItem
+		DataSet<ShippingPriorityItem> customerWithOrders = 
+				cust.join(or)
+					.where(0)
+					.equalTo(0)
+					.with(
+							new JoinFunction<Customer, Order, ShippingPriorityItem>() {
+								@Override
+								public ShippingPriorityItem join(Customer first, Order second) {
+									return new ShippingPriorityItem(0, 0.0, second.getOrderdate(),
+											second.getShippriority(), second.getOrderkey());
+								}
+							});
+		
+		// Join the last join result with Lineitems
+		DataSet<ShippingPriorityItem> joined = 
+				customerWithOrders.join(li)
+									.where(4)
+									.equalTo(0)
+									.with(
+											new JoinFunction<ShippingPriorityItem, Lineitem, ShippingPriorityItem>() {
+												@Override
+												public ShippingPriorityItem join(ShippingPriorityItem first, Lineitem second) {
+													first.setL_Orderkey(second.getOrderkey());
+													first.setRevenue(second.getExtendedprice() * (1 - second.getDiscount()));
+													return first;
+												}
+											});
+		
+		// Group by l_orderkey, o_orderdate and o_shippriority and compute revenue sum
+		joined = joined
+				.groupBy(0, 2, 3)
+				.aggregate(Aggregations.SUM, 1);
+		
+		// emit result
+		joined.writeAsCsv(outputPath, "\n", "|");
+		
+		// execute program
+		env.execute("TPCH Query 3 Example");
+		
+	}
+	
+	// *************************************************************************
+	//     DATA TYPES
+	// *************************************************************************
+	
+	public static class Lineitem extends Tuple4<Integer, Double, Double, String> {
+
+		public Integer getOrderkey() { return this.f0; }
+		public Double getDiscount() { return this.f2; }
+		public Double getExtendedprice() { return this.f1; }
+		public String getShipdate() { return this.f3; }
+	}
+
+	public static class Customer extends Tuple2<Integer, String> {
+		
+		public Integer getCustKey() { return this.f0; }
+		public String getMktsegment() { return this.f1; }
+	}
+
+	public static class Order extends Tuple3<Integer, String, Integer> {
+		
+		public Integer getOrderkey() { return this.f0; }
+		public String getOrderdate() { return this.f1; }
+		public Integer getShippriority() { return this.f2; }
+	}
+
+	public static class ShippingPriorityItem extends Tuple5<Integer, Double, String, Integer, Integer> {
+
+		public ShippingPriorityItem() { }
+
+		public ShippingPriorityItem(Integer l_orderkey, Double revenue,
+				String o_orderdate, Integer o_shippriority, Integer o_orderkey) {
+			this.f0 = l_orderkey;
+			this.f1 = revenue;
+			this.f2 = o_orderdate;
+			this.f3 = o_shippriority;
+			this.f4 = o_orderkey;
+		}
+		
+		public Integer getL_Orderkey() { return this.f0; }
+		public void setL_Orderkey(Integer l_orderkey) { this.f0 = l_orderkey; }
+		public Double getRevenue() { return this.f1; }
+		public void setRevenue(Double revenue) { this.f1 = revenue; }
+		
+		public String getOrderdate() { return this.f2; }
+		public Integer getShippriority() { return this.f3; }
+		public Integer getO_Orderkey() { return this.f4; }
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static String lineitemPath;
+	private static String customerPath;
+	private static String ordersPath;
+	private static String outputPath;
+	
+	private static boolean parseParameters(String[] programArguments) {
+		
+		if(programArguments.length > 0) {
+			if(programArguments.length == 4) {
+				lineitemPath = programArguments[0];
+				customerPath = programArguments[1];
+				ordersPath = programArguments[2];
+				outputPath = programArguments[3];
+			} else {
+				System.err.println("Usage: TPCHQuery3 <lineitem-csv path> <customer-csv path> <orders-csv path> <result path>");
+				return false;
+			}
+		} else {
+			System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
+								"  Due to legal restrictions, we can not ship generated data.\n" +
+								"  You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + 
+								"  Usage: TPCHQuery3 <lineitem-csv path> <customer-csv path> <orders-csv path> <result path>");
+			return false;
+		}
+		return true;
+	}
+	
+	private static DataSet<Lineitem> getLineitemDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(lineitemPath)
+					.fieldDelimiter('|')
+					.includeFields("1000011000100000")
+					.tupleType(Lineitem.class);
+	}
+	
+	private static DataSet<Customer> getCustomerDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(customerPath)
+					.fieldDelimiter('|')
+					.includeFields("10000010")
+					.tupleType(Customer.class);
+	}
+	
+	private static DataSet<Order> getOrdersDataSet(ExecutionEnvironment env) {
+		return env.readCsvFile(ordersPath)
+					.fieldDelimiter('|')
+					.includeFields("100010010")
+					.tupleType(Order.class);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
new file mode 100644
index 0000000..18de78b
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
@@ -0,0 +1,328 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.relational;
+
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.relational.util.WebLogData;
+import org.apache.flink.examples.java.relational.util.WebLogDataGenerator;
+
+/**
+ * This program processes web logs and relational data. 
+ * It implements the following relational query:
+ *
+ * <code><pre>
+ * SELECT 
+ *       r.pageURL, 
+ *       r.pageRank, 
+ *       r.avgDuration
+ * FROM documents d JOIN rankings r
+ *                  ON d.url = r.url
+ * WHERE CONTAINS(d.text, [keywords]) 
+ *       AND r.rank > [rank] 
+ *       AND NOT EXISTS 
+ *           (
+ *              SELECT * FROM Visits v
+ *              WHERE v.destUrl = d.url 
+ *                    AND v.visitDate < [date]
+ *           );
+ * </pre></code>
+ *
+ * <p>
+ * Input files are plain text CSV files using the pipe character ('|') as field separator.
+ * The tables referenced in the query can be generated using the {@link WebLogDataGenerator} and 
+ * have the following schemas
+ * <code><pre>
+ * CREATE TABLE Documents (
+ *                url VARCHAR(100) PRIMARY KEY,
+ *                contents TEXT );
+ *
+ * CREATE TABLE Rankings (
+ *                pageRank INT,
+ *                pageURL VARCHAR(100) PRIMARY KEY,
+ *                avgDuration INT );
+ *
+ * CREATE TABLE Visits (
+ *                sourceIP VARCHAR(16),
+ *                destURL VARCHAR(100),
+ *                visitDate DATE,
+ *                adRevenue FLOAT,
+ *                userAgent VARCHAR(64),
+ *                countryCode VARCHAR(3),
+ *                languageCode VARCHAR(6),
+ *                searchWord VARCHAR(32),
+ *                duration INT );
+ * </pre></code>
+ * 
+ * <p>
+ * Usage: <code>WebLogAnalysis &lt;documents path&gt; &lt;ranks path&gt; &lt;visits path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WebLogData}.
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> tuple data types
+ * <li> projection and join projection
+ * <li> the CoGroup transformation for an anti-join
+ * </ul>
+ * 
+ */
+@SuppressWarnings("serial")
+public class WebLogAnalysis {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataSet<Tuple2<String, String>> documents = getDocumentsDataSet(env);
+		DataSet<Tuple3<Integer, String, Integer>> ranks = getRanksDataSet(env);
+		DataSet<Tuple2<String, String>> visits = getVisitsDataSet(env);
+		
+		// Retain documents with keywords
+		DataSet<Tuple1<String>> filterDocs = documents
+				.filter(new FilterDocByKeyWords())
+				.project(0).types(String.class);
+
+		// Filter ranks by minimum rank
+		DataSet<Tuple3<Integer, String, Integer>> filterRanks = ranks
+				.filter(new FilterByRank());
+
+		// Filter visits by visit date
+		DataSet<Tuple1<String>> filterVisits = visits
+				.filter(new FilterVisitsByDate())
+				.project(0).types(String.class);
+
+		// Join the filtered documents and ranks, i.e., get all URLs with min rank and keywords
+		DataSet<Tuple3<Integer, String, Integer>> joinDocsRanks = 
+				filterDocs.join(filterRanks)
+							.where(0).equalTo(1)
+							.projectSecond(0,1,2)
+							.types(Integer.class, String.class, Integer.class);
+
+		// Anti-join urls with visits, i.e., retain all URLs which have NOT been visited in a certain time
+		DataSet<Tuple3<Integer, String, Integer>> result = 
+				joinDocsRanks.coGroup(filterVisits)
+								.where(1).equalTo(0)
+								.with(new AntiJoinVisits());
+
+		// emit result
+		if(fileOutput) {
+			result.writeAsCsv(outputPath, "\n", "|");
+		} else {
+			result.print();
+		}
+
+		// execute program
+		env.execute("WebLogAnalysis Example");
+		
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+	
+	/**
+	 * MapFunction that filters for documents that contain a certain set of
+	 * keywords.
+	 */
+	public static class FilterDocByKeyWords implements FilterFunction<Tuple2<String, String>> {
+
+		private static final String[] KEYWORDS = { " editors ", " oscillations " };
+
+		/**
+		 * Filters for documents that contain all of the given keywords and projects the records on the URL field.
+		 *
+		 * Output Format:
+		 * 0: URL
+		 * 1: DOCUMENT_TEXT
+		 */
+		@Override
+		public boolean filter(Tuple2<String, String> value) throws Exception {
+			// FILTER
+			// Only collect the document if all keywords are contained
+			String docText = value.f1;
+			for (String kw : KEYWORDS) {
+				if (!docText.contains(kw)) {
+					return false;
+				}
+			}
+			return true;
+		}
+	}
+
+	/**
+	 * MapFunction that filters for records where the rank exceeds a certain threshold.
+	 */
+	public static class FilterByRank implements FilterFunction<Tuple3<Integer, String, Integer>> {
+
+		private static final int RANKFILTER = 40;
+
+		/**
+		 * Filters for records of the rank relation where the rank is greater
+		 * than the given threshold.
+		 *
+		 * Output Format:
+		 * 0: RANK
+		 * 1: URL
+		 * 2: AVG_DURATION
+		 */
+		@Override
+		public boolean filter(Tuple3<Integer, String, Integer> value) throws Exception {
+			return (value.f0 > RANKFILTER);
+		}
+	}
+
+	/**
+	 * MapFunction that filters for records of the visits relation where the year
+	 * (from the date string) is equal to a certain value.
+	 */
+	public static class FilterVisitsByDate implements FilterFunction<Tuple2<String, String>> {
+
+		private static final int YEARFILTER = 2007;
+
+		/**
+		 * Filters for records of the visits relation where the year of visit is equal to a
+		 * specified value. The URL of all visit records passing the filter is emitted.
+		 *
+		 * Output Format:
+		 * 0: URL
+		 * 1: DATE
+		 */
+		@Override
+		public boolean filter(Tuple2<String, String> value) throws Exception {
+			// Parse date string with the format YYYY-MM-DD and extract the year
+			String dateString = value.f1;
+			int year = Integer.parseInt(dateString.substring(0,4));
+			return (year == YEARFILTER);
+		}
+	}
+
+
+	/**
+	 * CoGroupFunction that realizes an anti-join.
+	 * If the first input does not provide any pairs, all pairs of the second input are emitted.
+	 * Otherwise, no pair is emitted.
+	 */
+	public static class AntiJoinVisits implements CoGroupFunction<Tuple3<Integer, String, Integer>, Tuple1<String>, Tuple3<Integer, String, Integer>> {
+
+		/**
+		 * If the visit iterator is empty, all pairs of the rank iterator are emitted.
+		 * Otherwise, no pair is emitted.
+		 *
+		 * Output Format:
+		 * 0: RANK
+		 * 1: URL
+		 * 2: AVG_DURATION
+		 */
+		@Override
+		public void coGroup(Iterable<Tuple3<Integer, String, Integer>> ranks, Iterable<Tuple1<String>> visits, Collector<Tuple3<Integer, String, Integer>> out) {
+			// Check if there is a entry in the visits relation
+			if (!visits.iterator().hasNext()) {
+				for (Tuple3<Integer, String, Integer> next : ranks) {
+					// Emit all rank pairs
+					out.collect(next);
+				}
+			}
+		}
+	}
+
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean fileOutput = false;
+	private static String documentsPath;
+	private static String ranksPath;
+	private static String visitsPath;
+	private static String outputPath;
+	
+	private static boolean parseParameters(String[] args) {
+		
+		if(args.length > 0) {
+			fileOutput = true;
+			if(args.length == 4) {
+				documentsPath = args[0];
+				ranksPath = args[1];
+				visitsPath = args[2];
+				outputPath = args[3];
+			} else {
+				System.err.println("Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing WebLog Analysis example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  We provide a data generator to create synthetic input files for this program.");
+			System.out.println("  Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>");
+		}
+		return true;
+	}
+	
+	private static DataSet<Tuple2<String, String>> getDocumentsDataSet(ExecutionEnvironment env) {
+		// Create DataSet for documents relation (URL, Doc-Text)
+		if(fileOutput) {
+			return env.readCsvFile(documentsPath)
+						.fieldDelimiter('|')
+						.types(String.class, String.class);
+		} else {
+			return WebLogData.getDocumentDataSet(env);
+		}
+	}
+	
+	private static DataSet<Tuple3<Integer, String, Integer>> getRanksDataSet(ExecutionEnvironment env) {
+		// Create DataSet for ranks relation (Rank, URL, Avg-Visit-Duration)
+		if(fileOutput) {
+			return env.readCsvFile(ranksPath)
+						.fieldDelimiter('|')
+						.types(Integer.class, String.class, Integer.class);
+		} else {
+			return WebLogData.getRankDataSet(env);
+		}
+	}
+
+	private static DataSet<Tuple2<String, String>> getVisitsDataSet(ExecutionEnvironment env) {
+		// Create DataSet for visits relation (URL, Date)
+		if(fileOutput) {
+			return env.readCsvFile(visitsPath)
+						.fieldDelimiter('|')
+						.includeFields("011000000")
+						.types(String.class, String.class);
+		} else {
+			return WebLogData.getVisitDataSet(env);
+		}
+	}
+		
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java
new file mode 100644
index 0000000..9d8b463
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java
@@ -0,0 +1,428 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.relational.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * Provides the default data sets used for the Weblog Analysis example program.
+ * The default data sets are used, if no parameters are given to the program.
+ *
+ */
+public class WebLogData {
+
+	public static final Object [][] DOCUMENTS = {
+			new Object [] {"url_0","dolor ad amet enim laoreet nostrud veniam aliquip ex nonummy diam dolore tincidunt tation exerci exerci wisi dolor nostrud "},
+			new Object [] {"url_1","wisi minim adipiscing nibh adipiscing ut nibh Lorem Ut nonummy euismod nibh wisi sit consectetuer exerci sed aliquip aliquip dolore aliquam enim dolore veniam aliquam euismod suscipit ad adipiscing exerci aliquip consectetuer euismod aliquip ad exerci ex nibh ex erat exerci laoreet lobortis quis "},
+			new Object [] {"url_2","diam sed convection aliquip amet commodo nonummy sed sed commodo commodo diam commodo adipiscing ad exerci magna exerci tation quis lobortis "},
+			new Object [] {"url_3","exerci suscipit sed lobortis amet lobortis aliquip nibh nostrud ad convection commodo ad nibh sed minim amet ad ea ea "},
+			new Object [] {"url_4","sit enim dolor quis laoreet ullamcorper veniam adipiscing ex quis commodo "},
+			new Object [] {"url_5","elit aliquip ea nisl oscillations sit dolor ipsum tincidunt ullamcorper dolore enim adipiscing laoreet elit ea volutpat adipiscing ea nibh nostrud Ut aliquam veniam Lorem laoreet veniam aliquip "},
+			new Object [] {"url_6","consectetuer ad sed suscipit euismod aliquip quis ullamcorper oscillations tation consectetuer tation amet suscipit nibh enim nonummy veniam commodo commodo diam euismod dolor Ut aliquip diam ex ad nonummy ad tincidunt minim exerci consectetuer veniam convection aliquam ut ut Lorem euismod sed ipsum volutpat "},
+			new Object [] {"url_7","Ut volutpat veniam ut consectetuer diam ut aliquam dolor nostrud erat consectetuer adipiscing exerci consectetuer Ut ullamcorper suscipit aliquam sed dolor nisl "},
+			new Object [] {"url_8","suscipit amet wisi nisl veniam lobortis sit Lorem aliquam nostrud aliquam ipsum ut laoreet suscipit Lorem laoreet editors adipiscing ullamcorper veniam erat consectetuer ut lobortis dolore elit sed tincidunt ipsum tation ullamcorper nonummy adipiscing ex ad laoreet ipsum suscipit lobortis lobortis Ut nonummy adipiscing erat volutpat aliquam "},
+			new Object [] {"url_9","nonummy commodo tation editors ut quis sit quis lobortis ea dolore oscillations diam ad dolor lobortis nisl ad veniam ullamcorper quis magna volutpat sit ipsum consectetuer dolore exerci commodo magna erat enim ut suscipit "},
+			new Object [] {"url_10","amet erat magna consectetuer tation tation aliquip nibh aliquam sed adipiscing ut commodo ex erat tincidunt aliquam ipsum Ut Ut sit tincidunt adipiscing suscipit minim sed erat dolor consectetuer Lorem consectetuer Lorem amet nibh diam ea ex enim suscipit wisi dolor nonummy magna enim euismod ullamcorper ut suscipit adipiscing "},
+			new Object [] {"url_11","ex quis exerci tation diam elit nostrud nostrud ut ipsum elit amet diam laoreet amet consectetuer volutpat sed lobortis "},
+			new Object [] {"url_12","elit suscipit sit ullamcorper ut ad erat ut dolor nostrud quis nisl enim erat dolor convection ad minim ut veniam nostrud sed editors adipiscing volutpat Ut aliquip commodo sed euismod adipiscing erat adipiscing dolore nostrud minim sed lobortis ea diam "},
+			new Object [] {"url_13","enim ut quis commodo veniam minim erat lobortis ad diam ex dolor tincidunt exerci ut aliquip tincidunt minim ut magna sed enim wisi veniam oscillations Lorem consectetuer "},
+			new Object [] {"url_14","nibh ipsum ullamcorper volutpat ut wisi dolor quis amet euismod quis ipsum ipsum minim tation volutpat sit exerci volutpat amet nonummy euismod veniam consectetuer sit consectetuer tincidunt nibh aliquam lobortis tation veniam ut ullamcorper wisi magna Ut volutpat consectetuer erat quis dolore ea tation "},
+			new Object [] {"url_15","ad wisi sed enim aliquam oscillations nibh Lorem lobortis veniam nibh laoreet nonummy sed nibh Lorem adipiscing diam magna nostrud magna oscillations ut oscillations elit nostrud diam editors Lorem "},
+			new Object [] {"url_16","nostrud volutpat veniam exerci tincidunt nostrud quis elit ipsum ea nonummy volutpat dolor elit lobortis magna nisl ut ullamcorper magna Lorem exerci nibh nisl magna editors erat aliquam aliquam ullamcorper sit aliquam sit nostrud oscillations consectetuer adipiscing suscipit convection exerci ea ullamcorper ex nisl "},
+			new Object [] {"url_17","ad ex aliquam erat aliquam elit veniam laoreet ut amet amet nostrud ut adipiscing Ut Lorem suscipit ex magna ullamcorper aliquam ullamcorper ullamcorper amet amet commodo aliquam volutpat nonummy nonummy tincidunt amet tation tincidunt volutpat ut veniam nisl erat dolor enim nonummy nostrud adipiscing laoreet adipiscing "},
+			new Object [] {"url_18","lobortis ipsum ex tincidunt tincidunt editors euismod consectetuer ipsum adipiscing lobortis exerci adipiscing nonummy nisl dolore nonummy erat exerci nisl ut dolore wisi volutpat lobortis magna "},
+			new Object [] {"url_19","ipsum tation laoreet tation adipiscing wisi nibh diam Ut suscipit ad wisi "},
+			new Object [] {"url_20","diam Lorem enim wisi ad lobortis dolor Ut ipsum amet dolore consectetuer nisl exerci nisl nonummy minim Ut erat oscillations ut Lorem nostrud dolore Ut dolore exerci ad ipsum dolore ex dolore aliquip sed aliquam ex aliquip magna amet ex dolore oscillations aliquip tation magna Ut "},
+			new Object [] {"url_21","lobortis ut amet ex nisl ullamcorper tincidunt ut elit diam quis suscipit ad amet ipsum magna Ut ex tincidunt "},
+			new Object [] {"url_22","amet commodo nisl ad quis lobortis ut commodo sit ut erat exerci lobortis suscipit nibh ut nostrud ut adipiscing commodo commodo quis quis nostrud nisl ipsum nostrud laoreet Lorem nostrud erat nostrud amet consectetuer laoreet oscillations wisi sit magna nibh amet "},
+			new Object [] {"url_23","adipiscing suscipit suscipit aliquip suscipit consectetuer minim magna ea erat nibh sit suscipit sed dolor oscillations nonummy volutpat ut tincidunt "},
+			new Object [] {"url_24","commodo sed tincidunt aliquip aliquip dolore commodo nonummy sed erat ut ex exerci dolore adipiscing tincidunt ex diam amet aliquam "},
+			new Object [] {"url_25","consectetuer consectetuer exerci quis ea veniam aliquam laoreet minim ex "},
+			new Object [] {"url_26","dolor exerci euismod minim magna quis erat consectetuer sed ex erat dolore quis ut oscillations ullamcorper Lorem exerci ex nibh ut exerci ullamcorper veniam nibh ut commodo ut Ut nostrud tincidunt tincidunt ad dolore Lorem ea tation enim erat nibh ut ea nonummy sed sed wisi nisl dolore "},
+			new Object [] {"url_27","amet elit ea ea nostrud editors Ut nostrud amet laoreet adipiscing ut nisl nonummy tincidunt ea ipsum ex dolore dolore oscillations sit minim Ut wisi ut laoreet minim elit "},
+			new Object [] {"url_28","wisi exerci volutpat Ut nostrud euismod minim Ut sit euismod ut ea magna consectetuer nisl ad minim tation nisl adipiscing Lorem aliquam quis exerci erat minim aliquip sit Lorem wisi wisi ut "},
+			new Object [] {"url_29","amet sed laoreet amet aliquam minim enim tincidunt Lorem sit aliquip amet suscipit ut laoreet elit suscipit erat ut tincidunt suscipit ipsum sed euismod elit dolore euismod dolore ut dolor nostrud ipsum tincidunt commodo adipiscing aliquam ut wisi dolor dolor suscipit "},
+			new Object [] {"url_30","euismod Lorem ex tincidunt amet enim minim suscipit exerci diam veniam amet nostrud ea ea "},
+			new Object [] {"url_31","ex ipsum sit euismod euismod ullamcorper tincidunt ut wisi ea adipiscing sed diam tation ipsum dolor aliquam veniam nonummy aliquip aliquip Lorem ut minim nisl tation sit exerci ullamcorper Ut dolor euismod aliquam consectetuer ad nonummy commodo exerci "},
+			new Object [] {"url_32","volutpat ipsum lobortis nisl veniam minim adipiscing dolor editors quis nostrud amet nostrud "},
+			new Object [] {"url_33","commodo wisi aliquip ut aliquam sed nostrud ex diam ad nostrud enim ut amet enim ea ad sed tation nostrud suscipit ea magna magna Lorem amet lobortis ut quis nibh aliquam aliquam exerci aliquip lobortis consectetuer enim wisi ea nisl laoreet erat dolore "},
+			new Object [] {"url_34","tincidunt adipiscing enim tation nibh Ut dolore tincidunt tation laoreet suscipit minim aliquam volutpat laoreet suscipit tincidunt nibh ut ut sit nostrud nonummy tincidunt exerci sit ad sed consectetuer minim dolor dolore laoreet nostrud nibh laoreet ea adipiscing exerci dolore ipsum "},
+			new Object [] {"url_35","tation ut erat ut tation dolor Lorem laoreet Lorem elit adipiscing wisi aliquip nostrud elit Ut volutpat ea aliquam aliquip "},
+			new Object [] {"url_36","lobortis enim ullamcorper adipiscing consectetuer aliquip wisi enim minim Ut minim elit elit aliquam exerci ullamcorper amet lobortis adipiscing diam laoreet consectetuer nostrud diam diam amet ut enim ullamcorper aliquip diam ut nostrud diam magna amet nonummy commodo wisi enim ullamcorper suscipit euismod dolore tincidunt magna suscipit elit "},
+			new Object [] {"url_37","elit adipiscing nisl nisl ex aliquip nibh sed ut ad Lorem elit consectetuer ad volutpat lobortis amet veniam ipsum nibh ut consectetuer editors ad aliquam "},
+			new Object [] {"url_38","elit quis nibh adipiscing sit consectetuer ut euismod quis tincidunt quis nisl consectetuer dolor diam suscipit quis dolore Lorem suscipit nonummy sed ex "},
+			new Object [] {"url_39","nisl sit consectetuer elit oscillations enim ipsum enim nostrud adipiscing nostrud editors aliquam "},
+			new Object [] {"url_40","sed wisi dolor diam commodo ullamcorper commodo nostrud ullamcorper laoreet minim dolore suscipit laoreet tation aliquip "},
+			new Object [] {"url_41","ad consectetuer exerci nisl exerci amet enim diam lobortis Lorem ex volutpat volutpat nibh aliquam ut ullamcorper volutpat nostrud ut adipiscing ullamcorper "},
+			new Object [] {"url_42","minim laoreet tation magna veniam ut ea sit ipsum tincidunt Ut amet ex aliquip ex euismod exerci wisi elit editors ad amet veniam ad editors "},
+			new Object [] {"url_43","ut nisl ad ullamcorper nibh Ut editors exerci enim exerci ea laoreet veniam ea amet exerci volutpat amet ad "},
+			new Object [] {"url_44","volutpat tincidunt enim amet sed tincidunt consectetuer ullamcorper nisl Ut adipiscing tation ad ad amet nonummy elit erat nibh Lorem erat elit laoreet consectetuer sed aliquip nostrud "},
+			new Object [] {"url_45","sed aliquam ut ut consectetuer wisi euismod enim erat euismod quis exerci amet tation sit "},
+			new Object [] {"url_46","lobortis oscillations tation aliquam dolore Lorem aliquip tation exerci ullamcorper aliquam aliquip lobortis ex tation dolor ut ut sed suscipit nisl ullamcorper sed editors laoreet aliquip enim dolor veniam tincidunt sed euismod tation "},
+			new Object [] {"url_47","Lorem Lorem ut wisi ad ut tation consectetuer exerci convection tation ullamcorper sed dolore quis aliquam ipsum lobortis commodo nonummy "},
+			new Object [] {"url_48","laoreet minim veniam nisl elit sit amet commodo ex ullamcorper suscipit aliquip laoreet convection Ut ex minim aliquam "},
+			new Object [] {"url_49","lobortis nonummy minim amet sit veniam quis consectetuer tincidunt laoreet quis "},
+			new Object [] {"url_50","lobortis nisl commodo dolor amet nibh editors enim magna minim elit euismod diam laoreet laoreet ad minim sed ut Ut lobortis adipiscing quis sed ut aliquam oscillations exerci tation consectetuer lobortis elit tincidunt consectetuer minim amet dolore quis aliquam Ut exerci sed aliquam quis quis ullamcorper Ut ex tincidunt "},
+			new Object [] {"url_51","nostrud nisl ea erat ut suscipit Ut sit oscillations ullamcorper nonummy magna lobortis dolore editors tincidunt nostrud suscipit ex quis tation ut sit amet nostrud laoreet ex tincidunt "},
+			new Object [] {"url_52","ea tation commodo elit sed ex sed quis enim nisl magna laoreet adipiscing amet sit nostrud consectetuer nibh tincidunt veniam ex veniam euismod exerci sed dolore suscipit nisl tincidunt euismod quis Ut enim euismod dolor diam exerci magna exerci ut exerci nisl "},
+			new Object [] {"url_53","volutpat amet Ut lobortis dolor tation minim nonummy lobortis convection nostrud "},
+			new Object [] {"url_54","ullamcorper commodo Ut amet sit nostrud aliquam ad amet wisi enim nostrud ipsum nisl veniam erat aliquam ex aliquam dolor dolor ut consectetuer euismod exerci elit exerci Ut ea minim enim consectetuer ad consectetuer nonummy convection adipiscing ad ullamcorper lobortis nonummy laoreet nonummy aliquam ullamcorper ad nostrud amet "},
+			new Object [] {"url_55","wisi magna editors amet aliquam diam amet aliquip nisl consectetuer laoreet nonummy suscipit euismod diam enim tation elit ut lobortis quis euismod suscipit nostrud ea ea commodo lobortis dolore Ut nisl nostrud dolor laoreet euismod ea dolore aliquam ut Lorem exerci ex sit "},
+			new Object [] {"url_56","ex dolor veniam wisi laoreet ut exerci diam ad ex ut ut laoreet ut nisl ullamcorper nisl "},
+			new Object [] {"url_57","diam adipiscing Ut ut Lorem amet erat elit erat magna adipiscing euismod elit ullamcorper nostrud aliquam dolor ullamcorper sit tation tation "},
+			new Object [] {"url_58","laoreet convection veniam lobortis dolore ut nonummy commodo erat lobortis veniam nostrud dolore minim commodo ut consectetuer magna erat ea dolore Lorem suscipit ex ipsum exerci sed enim ea tation suscipit enim adipiscing "},
+			new Object [] {"url_59","amet ut ut Ut ad dolor quis ad magna exerci suscipit magna nibh commodo euismod amet euismod wisi diam suscipit dolore Lorem dolor ex amet exerci aliquip ut ut lobortis quis elit minim sed Lorem "},
+			new Object [] {"url_60","ut ut amet ullamcorper amet euismod dolor amet elit exerci adipiscing sed suscipit sed exerci wisi diam veniam wisi suscipit ut quis nibh ullamcorper ex quis magna dolore volutpat editors minim ut sit aliquip oscillations nisl ipsum "},
+			new Object [] {"url_61","nibh nostrud tincidunt lobortis adipiscing adipiscing ullamcorper ullamcorper ipsum nisl ullamcorper aliquip laoreet commodo ut tation wisi diam commodo aliquip commodo suscipit tincidunt volutpat elit enim laoreet ut nostrud ad nonummy ipsum "},
+			new Object [] {"url_62","Ut ut minim enim amet euismod erat elit commodo consectetuer Ut quis dolor ex diam quis wisi tation tincidunt laoreet volutpat "},
+			new Object [] {"url_63","ut erat volutpat euismod amet ea nonummy lobortis ut Ut ea veniam sed veniam nostrud "},
+			new Object [] {"url_64","tation dolor suscipit minim nisl wisi consectetuer aliquip tation Ut commodo ut dolore consectetuer elit wisi nisl ipsum "},
+			new Object [] {"url_65","ullamcorper nisl Lorem magna tation veniam aliquam diam amet euismod "},
+			new Object [] {"url_66","euismod aliquam tincidunt Ut volutpat ea lobortis sit ut volutpat ut lobortis ut lobortis ut nisl amet dolor sed ipsum enim ullamcorper diam euismod nostrud wisi erat quis diam nibh Ut dolore sed amet tation enim diam "},
+			new Object [] {"url_67","amet minim minim amet laoreet Lorem aliquam veniam elit volutpat magna adipiscing enim enim euismod laoreet sed ex sed aliquam ad ea ut adipiscing suscipit ex minim dolore minim ea laoreet nisl "},
+			new Object [] {"url_68","aliquam ea volutpat ut wisi tation tation nibh nisl erat laoreet ea volutpat dolor dolor aliquam exerci quis ullamcorper aliquam ut quis suscipit "},
+			new Object [] {"url_69","quis exerci ut aliquip wisi dolore magna nibh consectetuer magna tation ullamcorper lobortis sed amet adipiscing minim suscipit nibh nibh nostrud euismod enim "},
+			new Object [] {"url_70","tation enim consectetuer adipiscing wisi laoreet diam aliquip nostrud elit nostrud aliquip ea minim amet diam dolore "},
+			new Object [] {"url_71","consectetuer tincidunt nibh amet tation nonummy sit tation diam sed diam tation "},
+			new Object [] {"url_72","Lorem ut nostrud nonummy minim quis euismod lobortis nostrud nonummy adipiscing tincidunt consectetuer ut nibh ad suscipit dolor ut elit dolore amet ut quis tation ullamcorper nonummy laoreet ullamcorper aliquam dolore convection dolor tincidunt ut ullamcorper ex dolor suscipit erat oscillations ad "},
+			new Object [] {"url_73","elit Ut commodo ut ullamcorper ullamcorper ut euismod commodo diam aliquip suscipit consectetuer exerci tation nostrud ut wisi exerci sed ut elit sed volutpat Lorem nibh laoreet consectetuer ex Lorem elit aliquam commodo lobortis ad "},
+			new Object [] {"url_74","quis magna laoreet commodo aliquam nisl ullamcorper veniam tation wisi consectetuer commodo consectetuer ad dolore aliquam dolor elit amet sit amet nibh commodo erat veniam aliquip dolore ad magna ad ipsum Ut exerci ea volutpat nisl amet nostrud sit "},
+			new Object [] {"url_75","tincidunt suscipit sit aliquip aliquam adipiscing dolore exerci Ut suscipit ut sit laoreet suscipit wisi sit enim nonummy consectetuer dolore editors "},
+			new Object [] {"url_76","veniam ullamcorper tation sit suscipit dolor suscipit veniam sit Lorem quis sed nostrud ad tincidunt elit adipiscing "},
+			new Object [] {"url_77","volutpat sit amet veniam quis ipsum nibh elit enim commodo magna veniam magna convection "},
+			new Object [] {"url_78","tation dolore minim elit nisl volutpat tation laoreet enim nostrud exerci dolore tincidunt aliquip Lorem ipsum nostrud quis adipiscing ullamcorper erat lobortis tation commodo Ut ipsum commodo magna ad ipsum ut enim "},
+			new Object [] {"url_79","lobortis amet elit Lorem amet nonummy commodo tation ex ea amet Lorem ea nonummy commodo veniam volutpat nibh wisi ad ipsum euismod ea convection nostrud nisl erat veniam Ut aliquip ad aliquip editors wisi magna tation nostrud nonummy adipiscing ullamcorper aliquip "},
+			new Object [] {"url_80","tincidunt nostrud nostrud magna ea euismod ea consectetuer nisl exerci ea dolor nisl commodo ex erat ipsum exerci suscipit ad nisl ea nonummy suscipit adipiscing laoreet sit euismod nibh adipiscing sed minim commodo amet "},
+			new Object [] {"url_81","nostrud erat ut sed editors erat amet magna lobortis diam laoreet dolor amet nibh ut ipsum ipsum amet ut sed ut exerci elit suscipit wisi magna ut veniam nisl commodo enim adipiscing laoreet ad Lorem oscillations "},
+			new Object [] {"url_82","quis commodo nibh nibh volutpat suscipit dolore magna tincidunt nibh ut ad ullamcorper ullamcorper quis enim ad ut tation minim laoreet veniam dolor sed tincidunt exerci exerci nostrud ullamcorper amet ut ut ullamcorper "},
+			new Object [] {"url_83","sit suscipit volutpat elit tation elit sed sed dolor ex ex ipsum euismod laoreet magna lobortis ad "},
+			new Object [] {"url_84","lobortis ipsum euismod enim ea tation veniam tation oscillations aliquip consectetuer euismod ut sed lobortis tation oscillations commodo euismod laoreet suscipit amet elit ullamcorper volutpat aliquam ea enim ullamcorper consectetuer laoreet tation quis ut commodo erat euismod dolor laoreet ullamcorper laoreet "},
+			new Object [] {"url_85","adipiscing sit quis commodo consectetuer quis enim euismod exerci nonummy ea nostrud Ut veniam sit aliquip nisl enim "},
+			new Object [] {"url_86","nostrud dolore veniam veniam wisi aliquip adipiscing diam sed quis ullamcorper "},
+			new Object [] {"url_87","quis Lorem suscipit Ut nibh diam euismod consectetuer lobortis ipsum sed suscipit consectetuer euismod laoreet ut wisi nisl elit quis commodo adipiscing adipiscing suscipit aliquam nisl quis magna ipsum enim ad quis ea magna Lorem nibh ea "},
+			new Object [] {"url_88","euismod commodo sed tincidunt Ut veniam consectetuer quis erat ex ea erat laoreet commodo nibh minim "},
+			new Object [] {"url_89","tation diam editors Ut enim nibh Lorem volutpat quis diam suscipit exerci wisi ad "},
+			new Object [] {"url_90","volutpat editors ea nibh wisi ad amet volutpat nisl ullamcorper nibh volutpat minim ex ut sit veniam Lorem consectetuer quis ad sit suscipit volutpat wisi diam sed tincidunt ipsum minim convection ea diam oscillations quis lobortis "},
+			new Object [] {"url_91","enim minim nonummy ea minim euismod adipiscing editors volutpat magna sit magna ut ipsum ut "},
+			new Object [] {"url_92","nisl Ut commodo amet euismod lobortis ea ea wisi commodo Lorem sit ipsum volutpat nonummy exerci erat elit exerci magna ad erat enim laoreet quis nostrud wisi ut veniam amet ullamcorper lobortis ad suscipit volutpat veniam nostrud nibh quis ipsum dolore consectetuer veniam ipsum aliquip dolore sed laoreet ipsum "},
+			new Object [] {"url_93","nonummy aliquam ad lobortis Lorem erat ad tation Lorem exerci ex "},
+			new Object [] {"url_94","nonummy dolore commodo exerci ex quis ut suscipit elit laoreet sit tation magna veniam ea sit nonummy veniam Lorem quis nibh aliquip exerci amet ullamcorper adipiscing erat nisl editors diam commodo ad euismod adipiscing ea suscipit exerci aliquip volutpat tation enim volutpat sit "},
+			new Object [] {"url_95","sit suscipit oscillations ipsum nibh dolor ea dolore ea elit ipsum minim editors magna consectetuer ullamcorper commodo nonummy sit nostrud aliquip sit erat ullamcorper ullamcorper nibh veniam erat quis dolore nonummy "},
+			new Object [] {"url_96","nostrud quis ut volutpat magna ad quis adipiscing Lorem commodo exerci laoreet magna adipiscing erat quis wisi ea ea laoreet enim convection ad dolor nisl amet nibh aliquam adipiscing tincidunt minim diam Lorem commodo adipiscing volutpat "},
+			new Object [] {"url_97","laoreet laoreet suscipit nostrud dolore adipiscing volutpat Ut sed nisl diam ullamcorper ex ut ut dolor amet nostrud euismod dolore veniam veniam enim tation veniam ea minim minim volutpat tincidunt "},
+			new Object [] {"url_98","quis lobortis amet wisi nostrud ipsum aliquam convection tincidunt dolore ullamcorper nibh lobortis volutpat ea nostrud oscillations minim nonummy enim ad lobortis exerci ipsum ullamcorper nibh nonummy diam amet enim veniam ut nostrud "},
+			new Object [] {"url_99","aliquam wisi suscipit commodo diam amet amet magna nisl enim nostrud tation nisl nostrud nibh ut "}
+	};
+
+	public static final Object [][] RANKS = {
+			new Object [] {30,"url_0",43},
+			new Object [] {82,"url_1",39},
+			new Object [] {56,"url_2",31},
+			new Object [] {96,"url_3",36},
+			new Object [] {31,"url_4",36},
+			new Object [] {29,"url_5",6},
+			new Object [] {33,"url_6",48},
+			new Object [] {66,"url_7",40},
+			new Object [] {28,"url_8",51},
+			new Object [] {9,"url_9",4},
+			new Object [] {49,"url_10",24},
+			new Object [] {26,"url_11",12},
+			new Object [] {39,"url_12",46},
+			new Object [] {84,"url_13",53},
+			new Object [] {29,"url_14",50},
+			new Object [] {21,"url_15",12},
+			new Object [] {69,"url_16",34},
+			new Object [] {11,"url_17",38},
+			new Object [] {96,"url_18",13},
+			new Object [] {56,"url_19",48},
+			new Object [] {18,"url_20",36},
+			new Object [] {31,"url_21",21},
+			new Object [] {29,"url_22",11},
+			new Object [] {71,"url_23",30},
+			new Object [] {85,"url_24",48},
+			new Object [] {19,"url_25",45},
+			new Object [] {69,"url_26",9},
+			new Object [] {20,"url_27",51},
+			new Object [] {33,"url_28",46},
+			new Object [] {75,"url_29",38},
+			new Object [] {96,"url_30",51},
+			new Object [] {73,"url_31",40},
+			new Object [] {67,"url_32",16},
+			new Object [] {24,"url_33",24},
+			new Object [] {27,"url_34",35},
+			new Object [] {33,"url_35",35},
+			new Object [] {7,"url_36",22},
+			new Object [] {83,"url_37",41},
+			new Object [] {23,"url_38",49},
+			new Object [] {41,"url_39",33},
+			new Object [] {66,"url_40",38},
+			new Object [] {4,"url_41",52},
+			new Object [] {34,"url_42",4},
+			new Object [] {28,"url_43",12},
+			new Object [] {14,"url_44",14},
+			new Object [] {41,"url_45",11},
+			new Object [] {48,"url_46",37},
+			new Object [] {75,"url_47",41},
+			new Object [] {78,"url_48",3},
+			new Object [] {63,"url_49",28}
+	};
+
+
+	public static final Object [][] VISITS = {
+			new Object [] {"url_2","2003-12-17"},
+			new Object [] {"url_9","2008-11-11"},
+			new Object [] {"url_14","2003-11-5"},
+			new Object [] {"url_46","2009-2-16"},
+			new Object [] {"url_14","2004-11-9"},
+			new Object [] {"url_36","2001-3-9"},
+			new Object [] {"url_35","2006-8-13"},
+			new Object [] {"url_22","2008-1-18"},
+			new Object [] {"url_36","2002-3-9"},
+			new Object [] {"url_13","2007-7-17"},
+			new Object [] {"url_23","2009-6-16"},
+			new Object [] {"url_16","2000-7-15"},
+			new Object [] {"url_41","2002-5-10"},
+			new Object [] {"url_6","2004-11-9"},
+			new Object [] {"url_5","2003-6-7"},
+			new Object [] {"url_22","2002-11-5"},
+			new Object [] {"url_11","2007-7-21"},
+			new Object [] {"url_38","2009-12-2"},
+			new Object [] {"url_6","2004-11-2"},
+			new Object [] {"url_46","2000-6-4"},
+			new Object [] {"url_34","2003-9-2"},
+			new Object [] {"url_31","2008-2-24"},
+			new Object [] {"url_0","2003-2-2"},
+			new Object [] {"url_47","2003-7-8"},
+			new Object [] {"url_49","2009-9-13"},
+			new Object [] {"url_11","2003-4-2"},
+			new Object [] {"url_20","2000-6-18"},
+			new Object [] {"url_38","2000-2-22"},
+			new Object [] {"url_44","2009-2-17"},
+			new Object [] {"url_26","2000-6-21"},
+			new Object [] {"url_13","2000-11-25"},
+			new Object [] {"url_47","2005-4-19"},
+			new Object [] {"url_46","2008-1-7"},
+			new Object [] {"url_33","2004-12-24"},
+			new Object [] {"url_32","2009-2-8"},
+			new Object [] {"url_26","2000-9-21"},
+			new Object [] {"url_9","2002-8-18"},
+			new Object [] {"url_38","2002-11-27"},
+			new Object [] {"url_37","2008-2-26"},
+			new Object [] {"url_1","2007-3-22"},
+			new Object [] {"url_37","2002-3-20"},
+			new Object [] {"url_27","2008-11-12"},
+			new Object [] {"url_30","2000-12-16"},
+			new Object [] {"url_48","2000-12-17"},
+			new Object [] {"url_46","2008-4-16"},
+			new Object [] {"url_29","2006-3-9"},
+			new Object [] {"url_0","2007-7-26"},
+			new Object [] {"url_46","2009-12-15"},
+			new Object [] {"url_34","2002-2-13"},
+			new Object [] {"url_24","2009-3-1"},
+			new Object [] {"url_43","2007-11-4"},
+			new Object [] {"url_3","2004-2-16"},
+			new Object [] {"url_26","2000-10-26"},
+			new Object [] {"url_42","2004-7-14"},
+			new Object [] {"url_13","2004-9-10"},
+			new Object [] {"url_21","2000-2-21"},
+			new Object [] {"url_9","2006-6-5"},
+			new Object [] {"url_46","2001-12-17"},
+			new Object [] {"url_24","2006-12-8"},
+			new Object [] {"url_25","2006-9-2"},
+			new Object [] {"url_37","2002-6-26"},
+			new Object [] {"url_18","2006-6-2"},
+			new Object [] {"url_46","2003-5-24"},
+			new Object [] {"url_32","2000-10-17"},
+			new Object [] {"url_45","2002-1-12"},
+			new Object [] {"url_12","2005-12-13"},
+			new Object [] {"url_49","2009-3-9"},
+			new Object [] {"url_31","2001-9-19"},
+			new Object [] {"url_22","2002-7-9"},
+			new Object [] {"url_27","2005-2-3"},
+			new Object [] {"url_43","2008-7-15"},
+			new Object [] {"url_20","2000-3-23"},
+			new Object [] {"url_25","2002-5-8"},
+			new Object [] {"url_41","2004-4-27"},
+			new Object [] {"url_17","2008-7-17"},
+			new Object [] {"url_26","2009-12-16"},
+			new Object [] {"url_34","2006-2-10"},
+			new Object [] {"url_8","2009-4-14"},
+			new Object [] {"url_16","2000-2-24"},
+			new Object [] {"url_2","2009-2-10"},
+			new Object [] {"url_35","2003-2-24"},
+			new Object [] {"url_34","2008-3-16"},
+			new Object [] {"url_27","2005-1-5"},
+			new Object [] {"url_8","2008-12-10"},
+			new Object [] {"url_38","2009-2-11"},
+			new Object [] {"url_38","2006-11-3"},
+			new Object [] {"url_47","2003-2-13"},
+			new Object [] {"url_8","2008-11-17"},
+			new Object [] {"url_26","2009-5-11"},
+			new Object [] {"url_12","2007-11-26"},
+			new Object [] {"url_10","2003-1-13"},
+			new Object [] {"url_8","2005-9-23"},
+			new Object [] {"url_42","2001-4-5"},
+			new Object [] {"url_30","2009-12-10"},
+			new Object [] {"url_2","2003-1-3"},
+			new Object [] {"url_2","2009-2-19"},
+			new Object [] {"url_7","2000-6-25"},
+			new Object [] {"url_15","2004-9-26"},
+			new Object [] {"url_25","2009-10-5"},
+			new Object [] {"url_23","2009-8-9"},
+			new Object [] {"url_27","2004-4-3"},
+			new Object [] {"url_37","2008-6-9"},
+			new Object [] {"url_9","2002-5-25"},
+			new Object [] {"url_43","2009-5-18"},
+			new Object [] {"url_21","2008-4-19"},
+			new Object [] {"url_12","2001-12-25"},
+			new Object [] {"url_16","2006-9-25"},
+			new Object [] {"url_27","2002-1-2"},
+			new Object [] {"url_2","2009-1-21"},
+			new Object [] {"url_31","2009-3-20"},
+			new Object [] {"url_42","2002-3-1"},
+			new Object [] {"url_31","2001-11-26"},
+			new Object [] {"url_20","2003-5-15"},
+			new Object [] {"url_32","2004-1-22"},
+			new Object [] {"url_28","2008-9-16"},
+			new Object [] {"url_27","2006-7-3"},
+			new Object [] {"url_11","2008-12-26"},
+			new Object [] {"url_15","2004-8-16"},
+			new Object [] {"url_34","2002-10-5"},
+			new Object [] {"url_44","2000-2-15"},
+			new Object [] {"url_9","2000-10-23"},
+			new Object [] {"url_45","2005-4-24"},
+			new Object [] {"url_0","2006-8-7"},
+			new Object [] {"url_48","2003-8-7"},
+			new Object [] {"url_8","2007-12-13"},
+			new Object [] {"url_42","2003-8-2"},
+			new Object [] {"url_25","2008-3-5"},
+			new Object [] {"url_3","2007-3-9"},
+			new Object [] {"url_49","2003-10-7"},
+			new Object [] {"url_18","2007-12-6"},
+			new Object [] {"url_3","2006-7-5"},
+			new Object [] {"url_27","2000-9-14"},
+			new Object [] {"url_42","2002-10-20"},
+			new Object [] {"url_44","2007-1-13"},
+			new Object [] {"url_6","2003-1-21"},
+			new Object [] {"url_40","2009-10-20"},
+			new Object [] {"url_28","2009-6-17"},
+			new Object [] {"url_22","2000-2-17"},
+			new Object [] {"url_3","2005-1-15"},
+			new Object [] {"url_9","2008-12-9"},
+			new Object [] {"url_9","2005-2-19"},
+			new Object [] {"url_28","2000-4-22"},
+			new Object [] {"url_44","2001-9-9"},
+			new Object [] {"url_43","2008-6-21"},
+			new Object [] {"url_39","2008-5-9"},
+			new Object [] {"url_15","2006-9-15"},
+			new Object [] {"url_23","2001-12-18"},
+			new Object [] {"url_14","2002-5-23"},
+			new Object [] {"url_11","2007-7-11"},
+			new Object [] {"url_34","2000-12-8"},
+			new Object [] {"url_47","2005-7-3"},
+			new Object [] {"url_38","2004-3-26"},
+			new Object [] {"url_19","2003-9-14"},
+			new Object [] {"url_24","2007-7-16"},
+			new Object [] {"url_40","2008-8-21"},
+			new Object [] {"url_17","2007-12-4"},
+			new Object [] {"url_25","2006-6-24"},
+			new Object [] {"url_2","2000-10-8"},
+			new Object [] {"url_12","2008-6-10"},
+			new Object [] {"url_11","2004-11-24"},
+			new Object [] {"url_13","2005-11-3"},
+			new Object [] {"url_43","2005-1-2"},
+			new Object [] {"url_14","2008-6-12"},
+			new Object [] {"url_43","2001-8-27"},
+			new Object [] {"url_45","2000-3-3"},
+			new Object [] {"url_0","2006-9-27"},
+			new Object [] {"url_22","2007-12-18"},
+			new Object [] {"url_25","2006-4-4"},
+			new Object [] {"url_32","2001-6-25"},
+			new Object [] {"url_6","2007-6-9"},
+			new Object [] {"url_8","2009-10-3"},
+			new Object [] {"url_15","2003-2-23"},
+			new Object [] {"url_37","2000-5-6"},
+			new Object [] {"url_27","2004-3-21"},
+			new Object [] {"url_17","2005-6-20"},
+			new Object [] {"url_2","2004-2-27"},
+			new Object [] {"url_36","2005-3-16"},
+			new Object [] {"url_1","2009-12-3"},
+			new Object [] {"url_9","2004-4-27"},
+			new Object [] {"url_18","2009-5-26"},
+			new Object [] {"url_31","2000-9-21"},
+			new Object [] {"url_12","2008-9-25"},
+			new Object [] {"url_2","2004-2-16"},
+			new Object [] {"url_28","2008-11-12"},
+			new Object [] {"url_28","2001-6-26"},
+			new Object [] {"url_12","2006-3-15"},
+			new Object [] {"url_0","2009-3-1"},
+			new Object [] {"url_36","2006-10-13"},
+			new Object [] {"url_15","2004-11-5"},
+			new Object [] {"url_32","2008-2-11"},
+			new Object [] {"url_19","2009-8-3"},
+			new Object [] {"url_2","2006-8-6"},
+			new Object [] {"url_11","2009-10-13"},
+			new Object [] {"url_21","2002-9-14"},
+			new Object [] {"url_18","2000-11-2"},
+			new Object [] {"url_35","2006-5-15"},
+			new Object [] {"url_11","2006-2-18"},
+			new Object [] {"url_0","2001-4-25"},
+			new Object [] {"url_14","2009-4-8"},
+			new Object [] {"url_16","2009-4-7"}
+	};
+
+	public static DataSet<Tuple2<String, String>> getDocumentDataSet(ExecutionEnvironment env) {
+
+		List<Tuple2<String, String>> data = new ArrayList<Tuple2<String, String>>(100);
+		for (Object [] document : DOCUMENTS) {
+			data.add(new Tuple2<String, String>((String) document[0], (String) document[1]));
+		}
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<Integer, String, Integer>> getRankDataSet(ExecutionEnvironment env) {
+
+		List<Tuple3<Integer, String, Integer>> data = new ArrayList<Tuple3<Integer, String, Integer>>(100);
+		for (Object [] rank : RANKS) {
+			data.add(new Tuple3<Integer, String, Integer>((Integer) rank[0], (String) rank[1], (Integer) rank[2]));
+		}
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple2<String, String>> getVisitDataSet(ExecutionEnvironment env) {
+
+		List<Tuple2<String, String>> data = new ArrayList<Tuple2<String, String>>(100);
+
+		for (Object [] visit : VISITS) {
+			data.add(new Tuple2<String, String>((String) visit[0], (String) visit[1]));
+		}
+		return env.fromCollection(data);
+
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java
new file mode 100644
index 0000000..36d97a3
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.relational.util;
+
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Calendar;
+import java.util.Random;
+
+import org.apache.flink.examples.java.relational.WebLogAnalysis;
+
+/**
+ * Data generator for the {@link WebLogAnalysis} example program. 
+ *
+ */
+public class WebLogDataGenerator {
+
+	/**
+	 * Main method to generate data for the {@link WebLogAnalysis} example program.
+	 * <p>
+	 * The generator creates to files:
+	 * <ul>
+	 * <li><code>{tmp.dir}/documents</code> for the web documents
+	 * <li><code>{tmp.dir}/ranks</code> for the ranks of the web documents
+	 * <li><code>{tmp.dir}/visits</code> for the logged visits of web documents
+	 * </ul> 
+	 * 
+	 * @param args 
+	 * <ol>
+	 * <li>Int: Number of web documents
+	 * <li>Int: Number of visits
+	 * </ol>
+	 */
+	public static void main(String[] args) {
+
+		// parse parameters
+		if (args.length < 2) {
+			System.out.println("WebLogDataGenerator <numberOfDocuments> <numberOfVisits>");
+			System.exit(1);
+		}
+		
+		int noDocs = Integer.parseInt(args[0]);
+		int noVisits = Integer.parseInt(args[1]);
+		
+		String[] filterKWs = { "editors", "oscillations", "convection" };
+
+		String[] words = { "Lorem", "ipsum", "dolor", "sit", "amet",
+				"consectetuer", "adipiscing", "elit", "sed", "diam", "nonummy",
+				"nibh", "euismod", "tincidunt", "ut", "laoreet", "dolore",
+				"magna", "aliquam", "erat", "volutpat", "Ut", "wisi", "enim",
+				"ad", "minim", "veniam", "quis", "nostrud", "exerci", "tation",
+				"ullamcorper", "suscipit", "lobortis", "nisl", "ut", "aliquip",
+				"ex", "ea", "commodo" };
+
+		
+		final String outPath = System.getProperty("java.io.tmpdir");
+
+		System.out.println("Generating documents files...");
+		genDocs(noDocs, filterKWs, words, outPath + "/documents");
+		System.out.println("Generating ranks files...");
+		genRanks(noDocs, outPath + "/ranks");
+		System.out.println("Generating visits files...");
+		genVisits(noVisits, noDocs, outPath + "/visits");
+
+		System.out.println("Done!");
+	}
+
+	/**
+	 * Generates the files for the documents relation. The entries apply the
+	 * following format: <br />
+	 * <code>URL | Content</code>
+	 * 
+	 * @param noDocs
+	 *            Number of entries for the documents relation
+	 * @param filterKeyWords
+	 *            A list of keywords that should be contained
+	 * @param words
+	 *            A list of words to fill the entries
+	 * @param path
+	 *            Output path for the documents relation
+	 */
+	private static void genDocs(int noDocs, String[] filterKeyWords, String[] words, String path) {
+
+		Random rand = new Random(Calendar.getInstance().getTimeInMillis());
+
+		try {
+			FileWriter fw = new FileWriter(path);
+
+			for (int i = 0; i < noDocs; i++) {
+
+				int wordsInDoc = rand.nextInt(40) + 10;
+				// URL
+				StringBuilder doc = new StringBuilder("url_" + i + "|");
+				for (int j = 0; j < wordsInDoc; j++) {
+					if (rand.nextDouble() > 0.9) {
+						// Approx. every 10th word is a keyword
+						doc.append(filterKeyWords[rand
+								.nextInt(filterKeyWords.length)] + " ");
+					} else {
+						// Fills up the docs file(s) with random words
+						doc.append(words[rand.nextInt(words.length)] + " ");
+					}
+				}
+				doc.append("|\n");
+
+				fw.write(doc.toString());
+			}
+			fw.close();
+
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+	}
+
+	/**
+	 * Generates the files for the ranks relation. The ranks entries apply the
+	 * following format: <br />
+	 * <code>Rank | URL | Average Duration |\n</code>
+	 * 
+	 * @param noDocs
+	 *            Number of entries in the documents relation
+	 * @param path
+	 *            Output path for the ranks relation
+	 */
+	private static void genRanks(int noDocs, String path) {
+
+		Random rand = new Random(Calendar.getInstance().getTimeInMillis());
+
+		try {
+			FileWriter fw = new FileWriter(path);
+
+			for (int i = 0; i < noDocs; i++) {
+				// Rank
+				StringBuilder rank = new StringBuilder(rand.nextInt(100) + "|");
+				// URL
+				rank.append("url_" + i + "|");
+				// Average duration
+				rank.append(rand.nextInt(10) + rand.nextInt(50) + "|\n");
+
+				fw.write(rank.toString());
+			}
+			fw.close();
+
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+	}
+
+	/**
+	 * Generates the files for the visits relation. The visits entries apply the
+	 * following format:<br />
+	 * <code>IP Address | URL | Date (YYYY-MM-DD) | Misc. Data (e.g. User-Agent) |\n</code>
+	 * 
+	 * @param noVisits
+	 *            Number of entries for the visits relation
+	 * @param noDocs
+	 *            Number of entries in the documents relation
+	 * @param path
+	 *            Output path for the visits relation
+	 */
+	private static void genVisits(int noVisits, int noDocs, String path) {
+
+		Random rand = new Random(Calendar.getInstance().getTimeInMillis());
+
+		try {
+			FileWriter fw = new FileWriter(path);
+
+			for (int i = 0; i < noVisits; i++) {
+
+				int year = 2000 + rand.nextInt(10); // yearFilter 3
+				int month = rand.nextInt(12) + 1; // month between 1 and 12
+				int day = rand.nextInt(27) + 1; // day between 1 and 28
+
+				// IP address
+				StringBuilder visit = new StringBuilder(rand.nextInt(256) + "."
+						+ rand.nextInt(256) + "." + rand.nextInt(256) + "."
+						+ rand.nextInt(256) + "|");
+				// URL
+				visit.append("url_" + rand.nextInt(noDocs) + "|");
+				// Date (format: YYYY-MM-DD)
+				visit.append(year + "-" + month + "-" + day + "|");
+				// Miscellaneous data, e.g. User-Agent
+				visit.append("0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n");
+
+				fw.write(visit.toString());
+			}
+			fw.close();
+
+		} catch (IOException e) {
+			e.printStackTrace();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
new file mode 100644
index 0000000..b0c2c50
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.wordcount;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+
+/**
+ * Implements the "WordCount" program that computes a simple word occurrence histogram
+ * over text files. 
+ * 
+ * <p>
+ * The input is a plain text file with lines separated by newline characters.
+ * 
+ * <p>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * 
+ * <p>
+ * This example shows how to:
+ * <ul>
+ * <li>write a simple Flink program.
+ * <li>use Tuple data types.
+ * <li>write and use user-defined functions. 
+ * </ul>
+ * 
+ */
+@SuppressWarnings("serial")
+public class WordCount {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		// set up the execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// get input data
+		DataSet<String> text = getTextDataSet(env);
+		
+		DataSet<Tuple2<String, Integer>> counts = 
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				text.flatMap(new Tokenizer())
+				// group by the tuple field "0" and sum up tuple field "1"
+				.groupBy(0)
+				.sum(1);
+
+		// emit result
+		if(fileOutput) {
+			counts.writeAsCsv(outputPath, "\n", " ");
+		} else {
+			counts.print();
+		}
+		
+		// execute program
+		env.execute("WordCount Example");
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+	
+	/**
+	 * Implements the string tokenizer that splits sentences into words as a user-defined
+	 * FlatMapFunction. The function takes a line (String) and splits it into 
+	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
+	 */
+	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
+
+		@Override
+		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+			
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Tuple2<String, Integer>(token, 1));
+				}
+			}
+		}
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+	
+	private static boolean parseParameters(String[] args) {
+		
+		if(args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: WordCount <text path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing WordCount example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  Usage: WordCount <text path> <result path>");
+		}
+		return true;
+	}
+	
+	private static DataSet<String> getTextDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		} else {
+			// get default test text data
+			return WordCountData.getDefaultTextLineDataSet(env);
+		}
+	}
+}


[28/60] git commit: Added LinearRegression scala example. Removed old BGD example.

Posted by al...@apache.org.
Added LinearRegression scala example. Removed old BGD example.


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

Branch: refs/heads/master
Commit: 7fe9273f147d90bda4caed1c87847ee0e7a92c53
Parents: dbc680f
Author: mbalassi <ba...@gmail.com>
Authored: Sat Sep 13 15:15:17 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../java/ml/util/LinearRegressionData.java      |  70 +++---
 .../scala/datamining/BatchGradientDescent.scala |  83 -------
 .../examples/scala/ml/LinearRegression.scala    | 241 +++++++++++++++++++
 3 files changed, 278 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7fe9273f/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
index 5705bcd..31e71f5 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.example.java.ml.util;
 
 import org.apache.flink.api.java.DataSet;
@@ -24,45 +23,50 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.example.java.ml.LinearRegression.Data;
 import org.apache.flink.example.java.ml.LinearRegression.Params;
 
+import java.util.LinkedList;
+import java.util.List;
+
 /**
- * Provides the default data sets used for the Linear Regression example program.
- * The default data sets are used, if no parameters are given to the program.
- *
+ * Provides the default data sets used for the Linear Regression example
+ * program. The default data sets are used, if no parameters are given to the
+ * program.
  */
-public class LinearRegressionData{
+public class LinearRegressionData {
+
+	// We have the data as object arrays so that we can also generate Scala Data
+	// Sources from it.
+	public static final Object[][] PARAMS = new Object[][] { new Object[] {
+			0.0, 0.0 } };
 
-	public static DataSet<Params> getDefaultParamsDataSet(ExecutionEnvironment env){
+	public static final Object[][] DATA = new Object[][] {
+			new Object[] { 0.5, 1.0 }, new Object[] { 1.0, 2.0 },
+			new Object[] { 2.0, 4.0 }, new Object[] { 3.0, 6.0 },
+			new Object[] { 4.0, 8.0 }, new Object[] { 5.0, 10.0 },
+			new Object[] { 6.0, 12.0 }, new Object[] { 7.0, 14.0 },
+			new Object[] { 8.0, 16.0 }, new Object[] { 9.0, 18.0 },
+			new Object[] { 10.0, 20.0 }, new Object[] { -0.08, -0.16 },
+			new Object[] { 0.13, 0.26 }, new Object[] { -1.17, -2.35 },
+			new Object[] { 1.72, 3.45 }, new Object[] { 1.70, 3.41 },
+			new Object[] { 1.20, 2.41 }, new Object[] { -0.59, -1.18 },
+			new Object[] { 0.28, 0.57 }, new Object[] { 1.65, 3.30 },
+			new Object[] { -0.55, -1.08 } };
 
-		return env.fromElements(
-				new Params(0.0,0.0)
-				);
+	public static DataSet<Params> getDefaultParamsDataSet(
+			ExecutionEnvironment env) {
+		List<Params> paramsList = new LinkedList<Params>();
+		for (Object[] params : PARAMS) {
+			paramsList.add(new Params((Double) params[0], (Double) params[1]));
+		}
+		return env.fromCollection(paramsList);
 	}
 
-	public static DataSet<Data> getDefaultDataDataSet(ExecutionEnvironment env){
+	public static DataSet<Data> getDefaultDataDataSet(ExecutionEnvironment env) {
 
-		return env.fromElements(
-				new Data(0.5,1.0),
-				new Data(1.0,2.0),
-				new Data(2.0,4.0),
-				new Data(3.0,6.0),
-				new Data(4.0,8.0),
-				new Data(5.0,10.0),
-				new Data(6.0,12.0),
-				new Data(7.0,14.0),
-				new Data(8.0,16.0),
-				new Data(9.0,18.0),
-				new Data(10.0,20.0),
-				new Data(-0.08,-0.16),
-				new Data(0.13,0.26),
-				new Data(-1.17,-2.35),
-				new Data(1.72,3.45),
-				new Data(1.70,3.41),
-				new Data(1.20,2.41),
-				new Data(-0.59,-1.18),
-				new Data(0.28,0.57),
-				new Data(1.65,3.30),
-				new Data(-0.55,-1.08)
-				);
+		List<Data> dataList = new LinkedList<Data>();
+		for (Object[] data : DATA) {
+			dataList.add(new Data((Double) data[0], (Double) data[1]));
+		}
+		return env.fromCollection(dataList);
 	}
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7fe9273f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
deleted file mode 100644
index 50f01c9..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
+++ /dev/null
@@ -1,83 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.datamining
-//
-//import scala.math._
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//
-//abstract class BatchGradientDescent(eps: Double, eta: Double, lambda: Double, examplesInput: String, weightsInput: String, weightsOutput: String) extends Serializable {
-//  def computeGradient(example: Array[Double], weight: Array[Double]): (Double, Array[Double])
-//
-//  def updateWeight = (prev: (Int, Array[Double], Double), vg: ValueAndGradient) => {
-//    val (id, wOld, eta) = prev
-//    val ValueAndGradient(_, lossSum, gradSum) = vg
-//
-//    val delta = lossSum + lambda * wOld.norm
-//    val wNew = (wOld + (gradSum * eta)) * (1 - eta * lambda)
-//    (id, delta, wNew, eta * 0.9)
-//  }
-//
-//  class WeightVector(vector: Array[Double]) {
-//    def +(that: Array[Double]): Array[Double] = (vector zip that) map { case (x1, x2) => x1 + x2 }
-//    def -(that: Array[Double]): Array[Double] = (vector zip that) map { case (x1, x2) => x1 - x2 }
-//    def *(x: Double): Array[Double] = vector map { x * _ }
-//    def norm: Double = sqrt(vector map { x => x * x } reduce { _ + _ })
-//  }
-//
-//  implicit def array2WeightVector(vector: Array[Double]): WeightVector = new WeightVector(vector)
-//
-//  case class ValueAndGradient(id: Int, value: Double, gradient: Array[Double]) {
-//    def this(id: Int, vg: (Double, Array[Double])) = this(id, vg._1, vg._2)
-//    def +(that: ValueAndGradient) = ValueAndGradient(id, value + that.value, gradient + that.gradient)
-//  }
-//
-//  def readVector = (line: String) => {
-//    val Seq(id, vector @ _*) = line.split(',').toSeq
-//    id.toInt -> (vector map { _.toDouble } toArray)
-//  }
-//
-//  def formatOutput = (id: Int, vector: Array[Double]) => "%s,%s".format(id, vector.mkString(","))
-//
-//  def getPlan() = {
-//
-//    val examples = DataSource(examplesInput, DelimitedInputFormat(readVector))
-//    val weights = DataSource(weightsInput, DelimitedInputFormat(readVector))
-//
-//    def gradientDescent = (s: DataSetOLD[(Int, Array[Double])], ws: DataSetOLD[(Int, Array[Double], Double)]) => {
-//
-//      val lossesAndGradients = ws cross examples map { (w, ex) => new ValueAndGradient(w._1, computeGradient(ex._2, w._2)) }
-//      val lossAndGradientSums = lossesAndGradients groupBy { _.id } reduce (_ + _)
-//      val newWeights = ws join lossAndGradientSums where { _._1 } isEqualTo { _.id } map updateWeight
-//
-//      val s1 = newWeights map { case (wId, _, wNew, _) => (wId, wNew) } // updated solution elements
-//      val ws1 = newWeights filter { case (_, delta, _, _) => delta > eps } map { case (wId, _, wNew, etaNew) => (wId, wNew, etaNew) } // new workset
-//
-//      (s1, ws1)
-//    }
-//
-//    val newWeights = weights.iterateWithDelta(weights.map { case (id, w) => (id, w, eta) }, {_._1}, gradientDescent, 10)
-//
-//    val output = newWeights.write(weightsOutput, DelimitedOutputFormat(formatOutput.tupled))
-//    new ScalaPlan(Seq(output), "Batch Gradient Descent")
-//  }
-//}
-//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7fe9273f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
new file mode 100644
index 0000000..cf00d74
--- /dev/null
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
@@ -0,0 +1,241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.scala.ml
+
+import java.io.Serializable
+
+import org.apache.flink.api.common.functions._
+import org.apache.flink.api.scala._
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.example.java.clustering.util.KMeansData
+import org.apache.flink.example.java.ml.util.LinearRegressionData
+
+import scala.collection.JavaConverters._
+
+/**
+ * This example implements a basic Linear Regression  to solve the y = theta0 + theta1*x problem using batch gradient descent algorithm.
+ *
+ * <p>
+ * Linear Regression with BGD(batch gradient descent) algorithm is an iterative clustering algorithm and works as follows:<br>
+ * Giving a data set and target set, the BGD try to find out the best parameters for the data set to fit the target set.
+ * In each iteration, the algorithm computes the gradient of the cost function and use it to update all the parameters.
+ * The algorithm terminates after a fixed number of iterations (as in this implementation)
+ * With enough iteration, the algorithm can minimize the cost function and find the best parameters
+ * This is the Wikipedia entry for the <a href = "http://en.wikipedia.org/wiki/Linear_regression">Linear regression</a> and <a href = "http://en.wikipedia.org/wiki/Gradient_descent">Gradient descent algorithm</a>.
+ * 
+ * <p>
+ * This implementation works on one-dimensional data. And find the two-dimensional theta.<br>
+ * It find the best Theta parameter to fit the target.
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Data points are represented as two double values separated by a blank character. The first one represent the X(the training data) and the second represent the Y(target).
+ * Data points are separated by newline characters.<br>
+ * For example <code>"-0.02 -0.04\n5.3 10.6\n"</code> gives two data points (x=-0.02, y=-0.04) and (x=5.3, y=10.6).
+ * </ul>
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> Bulk iterations
+ * <li> Broadcast variables in bulk iterations
+ * <li> Custom Java objects (PoJos)
+ * </ul>
+ */
+object LinearRegression {
+
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	def main(args: Array[String]) {
+		if (!parseParameters(args)) {
+			return
+		}
+
+		val env = ExecutionEnvironment.getExecutionEnvironment
+		val data: DataSet[Data] = getDataSet(env)
+		val parameters: DataSet[Params] = getParamsDataSet(env)
+		val result = parameters.iterate(numIterations) { currentParameters =>
+			val newParameters = data
+				.map(new SubUpdate).withBroadcastSet(currentParameters, "parameters")
+				.reduce { (val1, val2) =>
+				val new_theta0: Double = val1._1.getTheta0 + val2._1.getTheta0
+				val new_theta1: Double = val1._1.getTheta1 + val2._1.getTheta1
+				val result: Params = new Params(new_theta0, new_theta1)
+				(result, val1._2 + val2._2)
+			}
+				.map { x => x._1.div(x._2) }
+			newParameters
+		}
+
+		if (fileOutput) {
+			result.writeAsText(outputPath)
+		}
+		else {
+			result.print
+		}
+		env.execute("Scala Linear Regression example")
+	}
+
+	// *************************************************************************
+	//     DATA TYPES
+	// *************************************************************************
+	/**
+	 * A simple data sample, x means the input, and y means the target.
+	 */
+	class Data extends Serializable {
+
+		def this(x: Double, y: Double) {
+			this()
+			this.x = x
+			this.y = y
+		}
+
+		override def toString: String = {
+			"(" + x + "|" + y + ")"
+		}
+
+		var x: Double = .0
+		var y: Double = .0
+	}
+
+	/**
+	 * A set of parameters -- theta0, theta1.
+	 */
+	class Params extends Serializable {
+
+		def this(x0: Double, x1: Double) {
+			this()
+			this.theta0 = x0
+			this.theta1 = x1
+		}
+
+		override def toString: String = {
+			theta0 + " " + theta1
+		}
+
+		def getTheta0: Double = {
+			theta0
+		}
+
+		def getTheta1: Double = {
+			theta1
+		}
+
+		def setTheta0(theta0: Double) {
+			this.theta0 = theta0
+		}
+
+		def setTheta1(theta1: Double) {
+			this.theta1 = theta1
+		}
+
+		def div(a: Integer): Params = {
+			this.theta0 = theta0 / a
+			this.theta1 = theta1 / a
+			return this
+		}
+
+		private var theta0: Double = .0
+		private var theta1: Double = .0
+	}
+
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Compute a single BGD type update for every parameters.
+	 */
+	class SubUpdate extends RichMapFunction[Data, Tuple2[Params, Integer]] {
+
+		private var parameters: Traversable[Params] = null
+		var parameter: Params = null
+		private var count: Int = 1
+
+		/** Reads the parameters from a broadcast variable into a collection. */
+		override def open(parameters: Configuration) {
+			this.parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala
+		}
+
+		def map(in: Data): Tuple2[Params, Integer] = {
+			for (p <- parameters) {
+				this.parameter = p
+			}
+			val theta_0: Double = parameter.getTheta0 - 0.01 * ((parameter.getTheta0 + (parameter.getTheta1 * in.x)) - in.y)
+			val theta_1: Double = parameter.getTheta1 - 0.01 * (((parameter.getTheta0 + (parameter.getTheta1 * in.x)) - in.y) * in.x)
+			new Tuple2[Params, Integer](new Params(theta_0, theta_1), count)
+		}
+	}
+
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	private var fileOutput: Boolean = false
+	private var dataPath: String = null
+	private var outputPath: String = null
+	private var numIterations: Int = 10
+
+	private def parseParameters(programArguments: Array[String]): Boolean = {
+		if (programArguments.length > 0) {
+			fileOutput = true
+			if (programArguments.length == 3) {
+				dataPath = programArguments(0)
+				outputPath = programArguments(1)
+				numIterations = Integer.parseInt(programArguments(2))
+			}
+			else {
+				System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>")
+				false
+			}
+		}
+		else {
+			System.out.println("Executing Linear Regression example with default parameters and built-in default data.")
+			System.out.println("  Provide parameters to read input data from files.")
+			System.out.println("  See the documentation for the correct format of input files.")
+			System.out.println("  We provide a data generator to create synthetic input files for this program.")
+			System.out.println("  Usage: LinearRegression <data path> <result path> <num iterations>")
+		}
+		true
+	}
+
+	private def getDataSet(env: ExecutionEnvironment): DataSet[Data] = {
+		if (fileOutput) {
+			env.readCsvFile[(Double, Double)](
+				dataPath,
+				fieldDelimiter = ' ',
+				includedFields = Array(0, 1))
+				.map { t => new Data(t._1, t._2) }
+		}
+		else {
+			val data = LinearRegressionData.DATA map {
+				case Array(x, y) => new Data(x.asInstanceOf[Double], y.asInstanceOf[Double])
+			}
+			env.fromCollection(data)
+		}
+	}
+
+	private def getParamsDataSet(env: ExecutionEnvironment): DataSet[Params] = {
+		val params = LinearRegressionData.PARAMS map {
+			case Array(x, y) => new Params(x.asInstanceOf[Double], y.asInstanceOf[Double])
+		}
+		env.fromCollection(params)
+	}
+}


[33/60] git commit: [scala] Add Field Name Key Support for Scala Case Classes

Posted by al...@apache.org.
[scala] Add Field Name Key Support for Scala Case Classes

This does not change the runtime behavious. The key field names are
mapped to tuple indices at pre-flight time.

Also extends tests to cover the new feature.


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

Branch: refs/heads/master
Commit: 83debdb3cb5b34bd1f7f395f061bd3be876db061
Parents: 299cef7
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 12 11:38:20 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../scala/graph/EnumTrianglesBasic.scala        |   7 +-
 .../examples/scala/graph/EnumTrianglesOpt.scala | 383 ++++++++++---------
 .../examples/scala/graph/PageRankBasic.scala    | 313 +++++++--------
 .../examples/scala/misc/PiEstimation.scala      |  23 +-
 .../org/apache/flink/api/scala/DataSet.scala    |  81 +++-
 .../apache/flink/api/scala/GroupedDataSet.scala | 118 +++---
 .../apache/flink/api/scala/coGroupDataSet.scala |  28 +-
 .../api/scala/codegen/TypeInformationGen.scala  |   5 +-
 .../apache/flink/api/scala/crossDataSet.scala   |   2 +-
 .../apache/flink/api/scala/joinDataSet.scala    |  14 +-
 .../org/apache/flink/api/scala/package.scala    |  15 +-
 .../scala/typeutils/ScalaTupleTypeInfo.scala    |  15 +-
 .../api/scala/unfinishedKeyPairOperation.scala  |  54 ++-
 .../scala/operators/AggregateOperatorTest.scala |  40 ++
 .../scala/operators/CoGroupOperatorTest.scala   |  65 ++++
 .../scala/operators/DistinctOperatorTest.scala  |  53 ++-
 .../api/scala/operators/GroupingTest.scala      |  51 ++-
 .../api/scala/operators/JoinOperatorTest.scala  |  90 ++++-
 18 files changed, 900 insertions(+), 457 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
index 8672b2c..0bf01b4 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
@@ -85,9 +85,9 @@ object EnumTrianglesBasic {
 		
 		val triangles = edgesById
 						// build triads
-						.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new TriadBuilder())
+						.groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
 						// filter triads
-						.join(edgesById).where(1,2).equalTo(0,1) { (t, _) => Some(t) }
+						.join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => Some(t) }
 		
 		// emit result
 		if (fileOutput) {
@@ -163,8 +163,7 @@ object EnumTrianglesBasic {
 
 	private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
 		if (fileOutput) {
-			env.readCsvFile[(Int, Int)](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1)).
-			map { x => new Edge(x._1, x._2) }
+			env.readCsvFile[Edge](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1))
 		} else {
 			val edges = EnumTrianglesData.EDGES.map{ case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int]) }
 			env.fromCollection(edges)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
index 65c8b3e..e198986 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
@@ -30,206 +30,215 @@ import scala.collection.mutable.MutableList
 /**
  * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
  * A triangle consists of three edges that connect three vertices with each other.
- * 
- * <p>
- * The basic algorithm works as follows: 
+ *
+ * The basic algorithm works as follows:
  * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
  * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
  * that closes the triangle.
- * 
- * <p>
- * For a group of <i>n</i> edges that share a common vertex, the number of built triads is quadratic <i>((n*(n-1))/2)</i>.
- * Therefore, an optimization of the algorithm is to group edges on the vertex with the smaller output degree to 
- * reduce the number of triads. 
+ *
+ * For a group of ''i'' edges that share a common vertex, the number of built triads is quadratic
+ * ''(n*(n-1))/2)''. Therefore, an optimization of the algorithm is to group edges on the vertex
+ * with the smaller output degree to reduce the number of triads.
  * This implementation extends the basic algorithm by computing output degrees of edge vertices and 
  * grouping on edges on the vertex with the smaller degree.
- * 
- * <p>
+ *
  * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Edges are represented as pairs for vertex IDs which are separated by space 
- * characters. Edges are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
- * that include a triangle
- * </ul>
+ *
+ *  - Edges are represented as pairs for vertex IDs which are separated by space
+ *    characters. Edges are separated by new-line characters.
+ *    For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) edges (1)-(2), (2)-(12),
+ *    (1)-(12), and (42)-(63) that include a triangle
+ *
  * <pre>
  *     (1)
  *     /  \
  *   (2)-(12)
  * </pre>
- * 
- * Usage: <code>EnumTriangleOpt &lt;edge path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link EnumTrianglesData}.
- * 
- * <p>
+ *
+ * Usage:
+ * {{{
+ *   EnumTriangleOpt <edge path> <result path>
+ * }}}
+ *
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.example.java.graph.util.EnumTrianglesData]].
+ *
  * This example shows how to use:
- * <ul>
- * <li>Custom Java objects which extend Tuple
- * <li>Group Sorting
- * </ul>
- * 
+ *
+ *  - Custom Java objects which extend Tuple
+ *  - Group Sorting
+ *
  */
 object EnumTrianglesOpt {
-	
-	def main(args: Array[String]) {
-		if (!parseParameters(args)) {
-			return
-		}
-
-		// set up execution environment
-		val env = ExecutionEnvironment.getExecutionEnvironment
-
-		// read input data
-		val edges = getEdgeDataSet(env)
-		
-		val edgesWithDegrees = edges
-								// duplicate and switch edges
-								.flatMap( e => Array(e, Edge(e.v2, e.v1)) )
-								// add degree of first vertex
-								.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new DegreeCounter())
-								// join degrees of vertices
-								.groupBy(0,2).reduce( (e1, e2) => 	if(e1.d2 == 0) 
-																		new EdgeWithDegrees(e1.v1, e1.d1, e1.v2, e2.d2)
-																	else
-																		new EdgeWithDegrees(e1.v1, e2.d1, e1.v2, e1.d2)
-													)
-													
-		// project edges by degrees, vertex with smaller degree comes first
-		val edgesByDegree = edgesWithDegrees
-								.map(e => if (e.d1 < e.d2) Edge(e.v1, e.v2) else Edge(e.v2, e.v1) )
-		// project edges by Id, vertex with smaller Id comes first
-		val edgesById = edgesByDegree
-								.map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) )
-		
-		val triangles = edgesByDegree
-						// build triads
-						.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new TriadBuilder())
-						// filter triads
-						.join(edgesById).where(1,2).equalTo(0,1) { (t, _) => Some(t) }
-		
-		// emit result
-		if (fileOutput) {
-			triangles.writeAsCsv(outputPath, "\n", " ")
-		} else {
-			triangles.print()
-		}
-		
-		// execute program
-		env.execute("TriangleEnumeration Example")
-	}
-
-	// *************************************************************************
-	//     USER DATA TYPES
-	// *************************************************************************
-
-	case class Edge(v1: Int, v2: Int) extends Serializable
-	case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
-	case class EdgeWithDegrees(v1: Int, d1: Int, v2: Int, d2: Int) extends Serializable
-	
-		
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Counts the number of edges that share a common vertex.
-	 * Emits one edge for each input edge with a degree annotation for the shared vertex.
-	 * For each emitted edge, the first vertex is the vertex with the smaller id.
-	 */
-	class DegreeCounter extends GroupReduceFunction[Edge, EdgeWithDegrees] {
-		
-		val vertices = MutableList[Integer]()
-		var groupVertex = 0
-		
-		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[EdgeWithDegrees]) = {
-			
-			// empty vertex list
-			vertices.clear
-			
-			// collect all vertices
-			for(e <- edges.asScala) {
-				groupVertex = e.v1
-				if(!vertices.contains(e.v2) && e.v1 != e.v2) {
-					vertices += e.v2
-				}
-			}
-			
-			// count vertices to obtain degree of groupVertex
-			val degree = vertices.length
-			
-			// create and emit edges with degrees
-			for(v <- vertices) {
-				if (v < groupVertex) {
-					out.collect(new EdgeWithDegrees(v, 0, groupVertex, degree))
-				} else {
-					out.collect(new EdgeWithDegrees(groupVertex, degree, v, 0))
-				}
-			}
-		}
-	}
-	
-	/**
-	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
-	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
-	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
-	 */
-	class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
-
-		val vertices = MutableList[Integer]()
-		
-		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
-			
-			// clear vertex list
-			vertices.clear
-
-			// build and emit triads
-			for(e <- edges.asScala) {
-			
-				// combine vertex with all previously read vertices
-				for(v <- vertices) {
-					out.collect(Triad(e.v1, v, e.v2))
-				}
-				vertices += e.v2
-			}
-		}
-	}
-
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-
-	private def parseParameters(args: Array[String]): Boolean = {
-		if (args.length > 0) {
-			fileOutput = true
-			if (args.length == 2) {
-				edgePath = args(0)
-				outputPath = args(1)
-			} else {
-				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>")
-				false
-			}
-		} else {
-			System.out.println("Executing Enum Triangles Basic example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>");
-		}
-		true
-	}
-
-	private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
-		if (fileOutput) {
-			env.readCsvFile[(Int, Int)](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1)).
-			map { x => new Edge(x._1, x._2) }
-		} else {
-			val edges = EnumTrianglesData.EDGES.map{ case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int]) }
-			env.fromCollection(edges)
-		}
-	}
-	
-	
-	private var fileOutput: Boolean = false
-	private var edgePath: String = null
-	private var outputPath: String = null
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    // set up execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    // read input data
+    val edges = getEdgeDataSet(env)
+
+    val edgesWithDegrees = edges
+      // duplicate and switch edges
+      .flatMap(e => Seq(e, Edge(e.v2, e.v1)))
+      // add degree of first vertex
+      .groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new DegreeCounter())
+      // join degrees of vertices
+      .groupBy("v1", "v2").reduce {
+        (e1, e2) =>
+          if (e1.d2 == 0) {
+            new EdgeWithDegrees(e1.v1, e1.d1, e1.v2, e2.d2)
+          } else {
+            new EdgeWithDegrees(e1.v1, e2.d1, e1.v2, e1.d2)
+          }
+      }
+
+    // project edges by degrees, vertex with smaller degree comes first
+    val edgesByDegree = edgesWithDegrees
+      .map(e => if (e.d1 <= e.d2) Edge(e.v1, e.v2) else Edge(e.v2, e.v1))
+    // project edges by Id, vertex with smaller Id comes first
+    val edgesById = edgesByDegree
+      .map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1))
+
+    val triangles = edgesByDegree
+      // build triads
+      .groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
+      // filter triads
+      .join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => Some(t)}
+
+    // emit result
+    if (fileOutput) {
+      triangles.writeAsCsv(outputPath, "\n", ",")
+    } else {
+      triangles.print()
+    }
+
+    // execute program
+    env.execute("TriangleEnumeration Example")
+  }
+
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+
+  case class Edge(v1: Int, v2: Int) extends Serializable
+
+  case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
+
+  case class EdgeWithDegrees(v1: Int, d1: Int, v2: Int, d2: Int) extends Serializable
+
+
+  // *************************************************************************
+  //     USER FUNCTIONS
+  // *************************************************************************
+
+  /**
+   * Counts the number of edges that share a common vertex.
+   * Emits one edge for each input edge with a degree annotation for the shared vertex.
+   * For each emitted edge, the first vertex is the vertex with the smaller id.
+   */
+  class DegreeCounter extends GroupReduceFunction[Edge, EdgeWithDegrees] {
+
+    val vertices = mutable.MutableList[Integer]()
+    var groupVertex = 0
+
+    override def reduce(edges: java.lang.Iterable[Edge], out: Collector[EdgeWithDegrees]) = {
+
+      // empty vertex list
+      vertices.clear()
+
+      // collect all vertices
+      for (e <- edges.asScala) {
+        groupVertex = e.v1
+        if (!vertices.contains(e.v2) && e.v1 != e.v2) {
+          vertices += e.v2
+        }
+      }
+
+      // count vertices to obtain degree of groupVertex
+      val degree = vertices.length
+
+      // create and emit edges with degrees
+      for (v <- vertices) {
+        if (v < groupVertex) {
+          out.collect(new EdgeWithDegrees(v, 0, groupVertex, degree))
+        } else {
+          out.collect(new EdgeWithDegrees(groupVertex, degree, v, 0))
+        }
+      }
+    }
+  }
+
+  /**
+   * Builds triads (triples of vertices) from pairs of edges that share a vertex.
+   * The first vertex of a triad is the shared vertex, the second and third vertex are ordered by
+   * vertexId.
+   * Assumes that input edges share the first vertex and are in ascending order of the second
+   * vertex.
+   */
+  class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
+
+    val vertices = mutable.MutableList[Integer]()
+
+    override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
+
+      // clear vertex list
+      vertices.clear()
+
+      // build and emit triads
+      for (e <- edges.asScala) {
+        // combine vertex with all previously read vertices
+        for (v <- vertices) {
+          out.collect(Triad(e.v1, v, e.v2))
+        }
+        vertices += e.v2
+      }
+    }
+  }
+
+  // *************************************************************************
+  //     UTIL METHODS
+  // *************************************************************************
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      fileOutput = true
+      if (args.length == 2) {
+        edgePath = args(0)
+        outputPath = args(1)
+      } else {
+        System.err.println("Usage: EnumTriangleOpt <edge path> <result path>")
+        false
+      }
+    } else {
+      System.out.println("Executing Enum Triangles Optimized example with built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>")
+    }
+    true
+  }
+
+  private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
+    if (fileOutput) {
+      env.readCsvFile[Edge](
+        edgePath,
+        fieldDelimiter = ' ',
+        includedFields = Array(0, 1))
+    } else {
+      val edges = EnumTrianglesData.EDGES.map {
+        case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int])}
+      env.fromCollection(edges)
+    }
+  }
+
+
+  private var fileOutput: Boolean = false
+  private var edgePath: String = null
+  private var outputPath: String = null
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
index e24727c..28a0e48 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
@@ -17,169 +17,184 @@
  */
 package org.apache.flink.examples.scala.graph
 
-import scala.collection.JavaConverters._
 import org.apache.flink.api.scala._
 import org.apache.flink.example.java.graph.util.PageRankData
-import org.apache.flink.util.Collector
 import org.apache.flink.api.java.aggregation.Aggregations.SUM
 
+import org.apache.flink.util.Collector
 
 /**
  * A basic implementation of the Page Rank algorithm using a bulk iteration.
  * 
- * <p>
- * This implementation requires a set of pages and a set of directed links as input and works as follows. <br> 
- * In each iteration, the rank of every page is evenly distributed to all pages it points to.
- * Each page collects the partial ranks of all pages that point to it, sums them up, and applies a dampening factor to the sum.
- * The result is the new rank of the page. A new iteration is started with the new ranks of all pages.
- * This implementation terminates after a fixed number of iterations.<br>
- * This is the Wikipedia entry for the <a href="http://en.wikipedia.org/wiki/Page_rank">Page Rank algorithm</a>. 
+ * This implementation requires a set of pages and a set of directed links as input and works as
+ * follows.
+ *
+ * In each iteration, the rank of every page is evenly distributed to all pages it points to. Each
+ * page collects the partial ranks of all pages that point to it, sums them up, and applies a
+ * dampening factor to the sum. The result is the new rank of the page. A new iteration is started
+ * with the new ranks of all pages. This implementation terminates after a fixed number of
+ * iterations. This is the Wikipedia entry for the
+ * [[http://en.wikipedia.org/wiki/Page_rank Page Rank algorithm]]
  * 
- * <p>
  * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Pages represented as an (long) ID separated by new-line characters.<br> 
- * For example <code>"1\n2\n12\n42\n63\n"</code> gives five pages with IDs 1, 2, 12, 42, and 63.
- * <li>Links are represented as pairs of page IDs which are separated by space 
- * characters. Links are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).<br>
- * For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
- * </ul>
- * 
- * <p>
- * Usage: <code>PageRankBasic &lt;pages path&gt; &lt;links path&gt; &lt;output path&gt; &lt;num pages&gt; &lt;num iterations&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link PageRankData} and 10 iterations.
+ *
+ *  - Pages represented as an (long) ID separated by new-line characters.
+ *    For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
+ *  - Links are represented as pairs of page IDs which are separated by space  characters. Links
+ *    are separated by new-line characters.
+ *    For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12),
+ *    (1)->(12), and (42)->(63). For this simple implementation it is required that each page has
+ *    at least one incoming and one outgoing link (a page can point to itself).
+ *
+ * Usage:
+ * {{{
+ *   PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>
+ * }}}
+ *
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.example.java.graph.util.PageRankData]] and 10 iterations.
  * 
- * <p>
  * This example shows how to use:
- * <ul>
- * <li>Bulk Iterations
- * <li>Default Join
- * <li>Configure user-defined functions using constructor parameters.
- * </ul> 
+ *
+ *  - Bulk Iterations
+ *  - Default Join
+ *  - Configure user-defined functions using constructor parameters.
  *
  */
 object PageRankBasic {
-	
-	private final val DAMPENING_FACTOR: Double = 0.85;
-	private final val EPSILON: Double = 0.0001;
-
-	def main(args: Array[String]) {
-		if (!parseParameters(args)) {
-			return
-		}
-		
-		// set up execution environment
-		val env = ExecutionEnvironment.getExecutionEnvironment
-		
-		// read input data
-		val pages = getPagesDataSet(env)
-		val links = getLinksDataSet(env)
-		
-		// assign initial ranks to pages
-		val pagesWithRanks = pages.map(p => Page(p, (1.0/numPages)))
-		
-		// build adjacency list from link input
-		val adjacencyLists = links
-								// initialize lists
-								.map( e => AdjacencyList(e.sourceId, Array[java.lang.Long](e.targetId) ))
-								// concatenate lists
-								.groupBy(0).reduce( (l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds))
-		
-		// start iteration
-		val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) {
-			currentRanks => 
-				val newRanks = currentRanks
-								// distribute ranks to target pages
-								.join(adjacencyLists).where(0).equalTo(0)
-								.flatMap { x => for(targetId <- x._2.targetIds) yield Page(targetId, (x._1.rank / x._2.targetIds.length))}
-								// collect ranks and sum them up
-								.groupBy(0).aggregate(SUM, 1)
-								// apply dampening factor
-								.map { p => Page(p.pageId, (p.rank * DAMPENING_FACTOR) + ((1 - DAMPENING_FACTOR) / numPages) ) }
-				
-				// terminate if no rank update was significant
-				val termination = currentRanks
-									.join(newRanks).where(0).equalTo(0)
-									// check for significant update
-									.filter( x => math.abs(x._1.rank - x._2.rank) > EPSILON )
-				
-				(newRanks, termination)
-		}
-		
-		val result = finalRanks;
-								
-		// emit result
-		if (fileOutput) {
-			result.writeAsCsv(outputPath, "\n", " ")
-		} else {
-			result.print()
-		}
-		
-		// execute program
-		env.execute("Basic PageRank Example")
-	}
-	
-	// *************************************************************************
-	//     USER TYPES
-	// *************************************************************************
-
-	case class Link(sourceId: Long, targetId: Long)
-	case class Page(pageId: java.lang.Long, rank: Double)
-	case class AdjacencyList(sourceId: java.lang.Long, targetIds: Array[java.lang.Long])
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-
-	private def parseParameters(args: Array[String]): Boolean = {
-		if (args.length > 0) {
-			fileOutput = true
-			if (args.length == 5) {
-				pagesInputPath = args(0)
-				linksInputPath = args(1)
-				outputPath = args(2)
-				numPages = args(3).toLong
-				maxIterations = args(4).toInt
-			} else {
-				System.err.println("Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
-				false
-			}
-		} else {
-			System.out.println("Executing PageRank Basic example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
-			
-			numPages = PageRankData.getNumberOfPages();
-		}
-		true
-	}
-
-	private def getPagesDataSet(env: ExecutionEnvironment): DataSet[Long] = {
-		if(fileOutput) {
-			env.readCsvFile[Tuple1[Long]](pagesInputPath, fieldDelimiter = ' ', lineDelimiter = "\n")
-				.map(x => x._1)
-		} else {
-			env.fromCollection(Seq.range(1, PageRankData.getNumberOfPages()+1))
-		}
-	}
-	
-	private def getLinksDataSet(env: ExecutionEnvironment): DataSet[Link] = {
-		if (fileOutput) {
-			env.readCsvFile[(Long, Long)](linksInputPath, fieldDelimiter = ' ', includedFields = Array(0, 1))
-				.map { x => Link(x._1, x._2) }
-		} else {
-			val edges = PageRankData.EDGES.map{ case Array(v1, v2) => Link(v1.asInstanceOf[Long], v2.asInstanceOf[Long]) }
-			env.fromCollection(edges)
-		}
-	}	
-	
-	private var fileOutput: Boolean = false
-	private var pagesInputPath: String = null
-	private var linksInputPath: String = null
-	private var outputPath: String = null
-	private var numPages: Long = 0;
-	private var maxIterations: Int = 10;
+
+  private final val DAMPENING_FACTOR: Double = 0.85
+  private final val EPSILON: Double = 0.0001
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    // set up execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    // read input data
+    val pages = getPagesDataSet(env)
+    val links = getLinksDataSet(env)
+
+    // assign initial ranks to pages
+    val pagesWithRanks = pages.map(p => Page(p, 1.0 / numPages))
+
+    // build adjacency list from link input
+    val adjacencyLists = links
+      // initialize lists
+      .map(e => AdjacencyList(e.sourceId, Array(e.targetId)))
+      // concatenate lists
+      .groupBy("sourceId").reduce((l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds))
+
+    // start iteration
+    val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) {
+      currentRanks =>
+        val newRanks = currentRanks
+          // distribute ranks to target pages
+          .join(adjacencyLists).where("pageId").equalTo("sourceId") {
+            (page, adjacent, out: Collector[Page]) =>
+            for (targetId <- adjacent.targetIds) {
+              out.collect(Page(targetId, page.rank / adjacent.targetIds.length))
+            }
+          }
+          // collect ranks and sum them up
+          .groupBy("pageId").aggregate(SUM, "rank")
+          // apply dampening factor
+          .map { p =>
+            Page(p.pageId, (p.rank * DAMPENING_FACTOR) + ((1 - DAMPENING_FACTOR) / numPages))
+          }
+
+        // terminate if no rank update was significant
+        val termination = currentRanks.join(newRanks).where("pageId").equalTo("pageId") {
+          (current, next) =>
+            // check for significant update
+            if (math.abs(current.rank - next.rank) > EPSILON) Some(1) else None
+        }
+
+        (newRanks, termination)
+    }
+
+    val result = finalRanks
+
+    // emit result
+    if (fileOutput) {
+      result.writeAsCsv(outputPath, "\n", " ")
+    } else {
+      result.print()
+    }
+
+    // execute program
+    env.execute("Basic PageRank Example")
+  }
+
+  // *************************************************************************
+  //     USER TYPES
+  // *************************************************************************
+
+  case class Link(sourceId: Long, targetId: Long)
+
+  case class Page(pageId: Long, rank: Double)
+
+  case class AdjacencyList(sourceId: Long, targetIds: Array[Long])
+
+  // *************************************************************************
+  //     UTIL METHODS
+  // *************************************************************************
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      fileOutput = true
+      if (args.length == 5) {
+        pagesInputPath = args(0)
+        linksInputPath = args(1)
+        outputPath = args(2)
+        numPages = args(3).toLong
+        maxIterations = args(4).toInt
+      } else {
+        System.err.println("Usage: PageRankBasic <pages path> <links path> <output path> <num " +
+          "pages> <num iterations>")
+        false
+      }
+    } else {
+      System.out.println("Executing PageRank Basic example with default parameters and built-in " +
+        "default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  Usage: PageRankBasic <pages path> <links path> <output path> <num " +
+        "pages> <num iterations>")
+
+      numPages = PageRankData.getNumberOfPages
+    }
+    true
+  }
+
+  private def getPagesDataSet(env: ExecutionEnvironment): DataSet[Long] = {
+    if (fileOutput) {
+      env.readCsvFile[Tuple1[Long]](pagesInputPath, fieldDelimiter = ' ', lineDelimiter = "\n")
+        .map(x => x._1)
+    } else {
+      env.generateSequence(1, 15)
+    }
+  }
+
+  private def getLinksDataSet(env: ExecutionEnvironment): DataSet[Link] = {
+    if (fileOutput) {
+      env.readCsvFile[Link](linksInputPath, fieldDelimiter = ' ',
+        includedFields = Array(0, 1))
+    } else {
+      val edges = PageRankData.EDGES.map { case Array(v1, v2) => Link(v1.asInstanceOf[Long],
+        v2.asInstanceOf[Long])}
+      env.fromCollection(edges)
+    }
+  }
+
+  private var fileOutput: Boolean = false
+  private var pagesInputPath: String = null
+  private var linksInputPath: String = null
+  private var outputPath: String = null
+  private var numPages: Long = 0
+  private var maxIterations: Int = 10
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
index d702f61..bb66b10 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
@@ -26,22 +26,23 @@ object PiEstimation {
 
     val numSamples: Long = if (args.length > 0) args(0).toLong else 1000000
 
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val env = ExecutionEnvironment.getExecutionEnvironment
 
     // count how many of the samples would randomly fall into
-    // the unit circle
+    // the upper right quadrant of the unit circle
     val count =
       env.generateSequence(1, numSamples)
-      .map (sample => {
-        val x = Math.random()
-        val y = Math.random()
-        if (x * x + y * y < 1) 1L else 0L
-      })
-      .reduce(_+_)
-
-    // the ratio of the unit circle surface to 4 times the unit square is pi
+        .map  { sample =>
+          val x = Math.random()
+          val y = Math.random()
+          if (x * x + y * y < 1) 1L else 0L
+        }
+        .reduce(_+_)
+
+    // ratio of samples in upper right quadrant vs total samples gives surface of upper
+    // right quadrant, times 4 gives surface of whole unit circle, i.e. PI
     val pi = count
-      .map (_ * 4.0 / numSamples)
+      .map ( _ * 4.0 / numSamples)
 
     println("We estimate Pi to be:")
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 041f269..8f14c0a 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -30,6 +30,7 @@ import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
 import org.apache.flink.api.java.operators._
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
 import org.apache.flink.api.scala.operators.{ScalaCsvOutputFormat, ScalaAggregateOperator}
+import org.apache.flink.api.scala.typeutils.ScalaTupleTypeInfo
 import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.core.fs.{FileSystem, Path}
 import org.apache.flink.types.TypeInformation
@@ -376,6 +377,25 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
   }
 
   /**
+   * Creates a new [[DataSet]] by aggregating the specified field using the given aggregation
+   * function. Since this is not a keyed DataSet the aggregation will be performed on the whole
+   * collection of elements.
+   *
+   * This only works on CaseClass DataSets.
+   */
+  def aggregate(agg: Aggregations, field: String): DataSet[T] = {
+    val fieldIndex = fieldNames2Indices(set.getType, Array(field))(0)
+
+    set match {
+      case aggregation: ScalaAggregateOperator[T] =>
+        aggregation.and(agg, fieldIndex)
+        wrap(aggregation)
+
+      case _ => wrap(new ScalaAggregateOperator[T](set, agg, fieldIndex))
+    }
+  }
+
+  /**
    * Syntactic sugar for [[aggregate]] with `SUM`
    */
   def sum(field: Int) = {
@@ -397,6 +417,27 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
   }
 
   /**
+   * Syntactic sugar for [[aggregate]] with `SUM`
+   */
+  def sum(field: String) = {
+    aggregate(Aggregations.SUM, field)
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MAX`
+   */
+  def max(field: String) = {
+    aggregate(Aggregations.MAX, field)
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MIN`
+   */
+  def min(field: String) = {
+    aggregate(Aggregations.MIN, field)
+  }
+
+  /**
    * Creates a new [[DataSet]] by merging the elements of this DataSet using an associative reduce
    * function.
    */
@@ -486,7 +527,7 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    * Creates a new DataSet containing the distinct elements of this DataSet. The decision whether
    * two elements are distinct or not is made based on only the specified tuple fields.
    *
-   * This only works if this DataSet contains Tuples.
+   * This only works on tuple DataSets.
    */
   def distinct(fields: Int*): DataSet[T] = {
     wrap(new DistinctOperator[T](
@@ -496,6 +537,19 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
 
   /**
    * Creates a new DataSet containing the distinct elements of this DataSet. The decision whether
+   * two elements are distinct or not is made based on only the specified fields.
+   *
+   * This only works on CaseClass DataSets
+   */
+  def distinct(firstField: String, otherFields: String*): DataSet[T] = {
+    val fieldIndices = fieldNames2Indices(set.getType, firstField +: otherFields.toArray)
+    wrap(new DistinctOperator[T](
+      set,
+      new Keys.FieldPositionKeys[T](fieldIndices, set.getType, true)))
+  }
+
+  /**
+   * Creates a new DataSet containing the distinct elements of this DataSet. The decision whether
    * two elements are distinct or not is made based on all tuple fields.
    *
    * This only works if this DataSet contains Tuples.
@@ -539,6 +593,23 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
       new Keys.FieldPositionKeys[T](fields.toArray, set.getType,false))
   }
 
+  /**
+   * Creates a [[GroupedDataSet]] which provides operations on groups of elements. Elements are
+   * grouped based on the given fields.
+   *
+   * This will not create a new DataSet, it will just attach the field names which will be
+   * used for grouping when executing a grouped operation.
+   *
+   * This only works on CaseClass DataSets.
+   */
+  def groupBy(firstField: String, otherFields: String*): GroupedDataSet[T] = {
+    val fieldIndices = fieldNames2Indices(set.getType, firstField +: otherFields.toArray)
+
+    new GroupedDataSetImpl[T](
+      set,
+      new Keys.FieldPositionKeys[T](fieldIndices, set.getType,false))
+  }
+
   //	public UnsortedGrouping<T> groupBy(String... fields) {
   //		new UnsortedGrouping<T>(this, new Keys.ExpressionKeys<T>(fields, getType()));
   //	}
@@ -587,21 +658,21 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    * }}}
    */
   def join[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperationImpl(this.set, other.set, JoinHint.OPTIMIZER_CHOOSES)
+    new UnfinishedJoinOperationImpl(this, other, JoinHint.OPTIMIZER_CHOOSES)
 
   /**
    * Special [[join]] operation for explicitly telling the system that the right side is assumed
    * to be a lot smaller than the left side of the join.
    */
   def joinWithTiny[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperationImpl(this.set, other.set, JoinHint.BROADCAST_HASH_SECOND)
+    new UnfinishedJoinOperationImpl(this, other, JoinHint.BROADCAST_HASH_SECOND)
 
   /**
    * Special [[join]] operation for explicitly telling the system that the left side is assumed
    * to be a lot smaller than the right side of the join.
    */
   def joinWithHuge[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
-    new UnfinishedJoinOperationImpl(this.set, other.set, JoinHint.BROADCAST_HASH_FIRST)
+    new UnfinishedJoinOperationImpl(this, other, JoinHint.BROADCAST_HASH_FIRST)
 
   // --------------------------------------------------------------------------------------------
   //  Co-Group
@@ -641,7 +712,7 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    * }}}
    */
   def coGroup[O: ClassTag](other: DataSet[O]): UnfinishedCoGroupOperation[T, O] =
-    new UnfinishedCoGroupOperationImpl(this.set, other.set)
+    new UnfinishedCoGroupOperationImpl(this, other)
 
   // --------------------------------------------------------------------------------------------
   //  Cross

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
index dfd5cf0..a7ca821 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
@@ -19,6 +19,7 @@ package org.apache.flink.api.scala
 
 import org.apache.flink.api.common.InvalidProgramException
 import org.apache.flink.api.scala.operators.ScalaAggregateOperator
+import org.apache.flink.api.scala.typeutils.ScalaTupleTypeInfo
 
 import scala.collection.JavaConverters._
 
@@ -45,11 +46,21 @@ trait GroupedDataSet[T] {
 
   /**
    * Adds a secondary sort key to this [[GroupedDataSet]]. This will only have an effect if you
-   * use one of the group-at-a-time, i.e. `reduceGroup`
+   * use one of the group-at-a-time, i.e. `reduceGroup`.
+   *
+   * This only works on Tuple DataSets.
    */
   def sortGroup(field: Int, order: Order): GroupedDataSet[T]
 
   /**
+   * Adds a secondary sort key to this [[GroupedDataSet]]. This will only have an effect if you
+   * use one of the group-at-a-time, i.e. `reduceGroup`.
+   *
+   * This only works on CaseClass DataSets.
+   */
+  def sortGroup(field: String, order: Order): GroupedDataSet[T]
+
+  /**
    * Creates a new [[DataSet]] by aggregating the specified tuple field using the given aggregation
    * function. Since this is a keyed DataSet the aggregation will be performed on groups of
    * tuples with the same key.
@@ -59,6 +70,15 @@ trait GroupedDataSet[T] {
   def aggregate(agg: Aggregations, field: Int): DataSet[T]
 
   /**
+   * Creates a new [[DataSet]] by aggregating the specified field using the given aggregation
+   * function. Since this is a keyed DataSet the aggregation will be performed on groups of
+   * elements with the same key.
+   *
+   * This only works on CaseClass DataSets.
+   */
+  def aggregate(agg: Aggregations, field: String): DataSet[T]
+
+  /**
    * Syntactic sugar for [[aggregate]] with `SUM`
    */
   def sum(field: Int): DataSet[T]
@@ -74,6 +94,21 @@ trait GroupedDataSet[T] {
   def min(field: Int): DataSet[T]
 
   /**
+   * Syntactic sugar for [[aggregate]] with `SUM`
+   */
+  def sum(field: String): DataSet[T]
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MAX`
+   */
+  def max(field: String): DataSet[T]
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MIN`
+   */
+  def min(field: String): DataSet[T]
+
+  /**
    * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
    * using an associative reduce function.
    */
@@ -124,14 +159,10 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
   private val groupSortKeyPositions = mutable.MutableList[Int]()
   private val groupSortOrders = mutable.MutableList[Order]()
 
-  /**
-   * Adds a secondary sort key to this [[GroupedDataSet]]. This will only have an effect if you
-   * use one of the group-at-a-time, i.e. `reduceGroup`
-   */
   def sortGroup(field: Int, order: Order): GroupedDataSet[T] = {
     if (!set.getType.isTupleType) {
       throw new InvalidProgramException("Specifying order keys via field positions is only valid " +
-        "for tuple data types")
+        "for tuple data types.")
     }
     if (field >= set.getType.getArity) {
       throw new IllegalArgumentException("Order key out of tuple bounds.")
@@ -141,10 +172,14 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
     this
   }
 
-  /**
-   * Creates a [[SortedGrouping]] if any secondary sort fields were specified. Otherwise, just
-   * create an [[UnsortedGrouping]].
-   */
+  def sortGroup(field: String, order: Order): GroupedDataSet[T] = {
+    val fieldIndex = fieldNames2Indices(set.getType, Array(field))(0)
+
+    groupSortKeyPositions += fieldIndex
+    groupSortOrders += order
+    this
+  }
+
   private def maybeCreateSortedGrouping(): Grouping[T] = {
     if (groupSortKeyPositions.length > 0) {
       val grouping = new SortedGrouping[T](set, keys, groupSortKeyPositions(0), groupSortOrders(0))
@@ -161,13 +196,18 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
   /** Convenience methods for creating the [[UnsortedGrouping]] */
   private def createUnsortedGrouping(): Grouping[T] = new UnsortedGrouping[T](set, keys)
 
-  /**
-   * Creates a new [[DataSet]] by aggregating the specified tuple field using the given aggregation
-   * function. Since this is a keyed DataSet the aggregation will be performed on groups of
-   * tuples with the same key.
-   *
-   * This only works on Tuple DataSets.
-   */
+  def aggregate(agg: Aggregations, field: String): DataSet[T] = {
+    val fieldIndex = fieldNames2Indices(set.getType, Array(field))(0)
+
+    set match {
+      case aggregation: ScalaAggregateOperator[T] =>
+        aggregation.and(agg, fieldIndex)
+        wrap(aggregation)
+
+      case _ => wrap(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, fieldIndex))
+    }
+  }
+
   def aggregate(agg: Aggregations, field: Int): DataSet[T] = set match {
     case aggregation: ScalaAggregateOperator[T] =>
       aggregation.and(agg, field)
@@ -176,31 +216,30 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
     case _ => wrap(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, field))
   }
 
-  /**
-   * Syntactic sugar for [[aggregate]] with `SUM`
-   */
   def sum(field: Int): DataSet[T] = {
     aggregate(Aggregations.SUM, field)
   }
 
-  /**
-   * Syntactic sugar for [[aggregate]] with `MAX`
-   */
   def max(field: Int): DataSet[T] = {
     aggregate(Aggregations.MAX, field)
   }
 
-  /**
-   * Syntactic sugar for [[aggregate]] with `MIN`
-   */
   def min(field: Int): DataSet[T] = {
     aggregate(Aggregations.MIN, field)
   }
 
-  /**
-   * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
-   * using an associative reduce function.
-   */
+  def sum(field: String): DataSet[T] = {
+    aggregate(Aggregations.SUM, field)
+  }
+
+  def max(field: String): DataSet[T] = {
+    aggregate(Aggregations.MAX, field)
+  }
+
+  def min(field: String): DataSet[T] = {
+    aggregate(Aggregations.MIN, field)
+  }
+
   def reduce(fun: (T, T) => T): DataSet[T] = {
     Validate.notNull(fun, "Reduce function must not be null.")
     val reducer = new ReduceFunction[T] {
@@ -211,20 +250,11 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
     wrap(new ReduceOperator[T](createUnsortedGrouping(), reducer))
   }
 
-  /**
-   * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
-   * using an associative reduce function.
-   */
   def reduce(reducer: ReduceFunction[T]): DataSet[T] = {
     Validate.notNull(reducer, "Reduce function must not be null.")
     wrap(new ReduceOperator[T](createUnsortedGrouping(), reducer))
   }
 
-  /**
-   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
-   * of elements to the group reduce function. The function must output one element. The
-   * concatenation of those will form the resulting [[DataSet]].
-   */
   def reduceGroup[R: TypeInformation: ClassTag](
                                                  fun: (TraversableOnce[T]) => R): DataSet[R] = {
     Validate.notNull(fun, "Group reduce function must not be null.")
@@ -238,11 +268,6 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
         implicitly[TypeInformation[R]], reducer))
   }
 
-  /**
-   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
-   * of elements to the group reduce function. The function can output zero or more elements using
-   * the [[Collector]]. The concatenation of the emitted values will form the resulting [[DataSet]].
-   */
   def reduceGroup[R: TypeInformation: ClassTag](
                                                  fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
     Validate.notNull(fun, "Group reduce function must not be null.")
@@ -256,11 +281,6 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
         implicitly[TypeInformation[R]], reducer))
   }
 
-  /**
-   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
-   * of elements to the [[GroupReduceFunction]]. The function can output zero or more elements. The
-   * concatenation of the emitted values will form the resulting [[DataSet]].
-   */
   def reduceGroup[R: TypeInformation: ClassTag](reducer: GroupReduceFunction[T, R]): DataSet[R] = {
     Validate.notNull(reducer, "GroupReduce function must not be null.")
     wrap(

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
index 05f9917..f936b43 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
@@ -94,8 +94,8 @@ trait CoGroupDataSet[T, O] extends DataSet[(Array[T], Array[O])] {
  */
 private[flink] class CoGroupDataSetImpl[T, O](
     coGroupOperator: CoGroupOperator[T, O, (Array[T], Array[O])],
-    thisSet: JavaDataSet[T],
-    otherSet: JavaDataSet[O],
+    thisSet: DataSet[T],
+    otherSet: DataSet[O],
     thisKeys: Keys[T],
     otherKeys: Keys[O]) extends DataSet(coGroupOperator) with CoGroupDataSet[T, O] {
 
@@ -107,7 +107,7 @@ private[flink] class CoGroupDataSetImpl[T, O](
         fun(left.iterator.asScala, right.iterator.asScala) map { out.collect(_) }
       }
     }
-    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet, otherSet, thisKeys,
+    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet.set, otherSet.set, thisKeys,
       otherKeys, coGrouper, implicitly[TypeInformation[R]])
     wrap(coGroupOperator)
   }
@@ -120,14 +120,14 @@ private[flink] class CoGroupDataSetImpl[T, O](
         fun(left.iterator.asScala, right.iterator.asScala, out)
       }
     }
-    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet, otherSet, thisKeys,
+    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet.set, otherSet.set, thisKeys,
       otherKeys, coGrouper, implicitly[TypeInformation[R]])
     wrap(coGroupOperator)
   }
 
   def apply[R: TypeInformation: ClassTag](joiner: CoGroupFunction[T, O, R]): DataSet[R] = {
     Validate.notNull(joiner, "CoGroup function must not be null.")
-    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet, otherSet, thisKeys,
+    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet.set, otherSet.set, thisKeys,
       otherKeys, joiner, implicitly[TypeInformation[R]])
     wrap(coGroupOperator)
   }
@@ -153,8 +153,8 @@ trait UnfinishedCoGroupOperation[T, O]
  * i.e. the parameters of the constructor, hidden.
  */
 private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
-    leftSet: JavaDataSet[T],
-    rightSet: JavaDataSet[O])
+    leftSet: DataSet[T],
+    rightSet: DataSet[O])
   extends UnfinishedKeyPairOperation[T, O, CoGroupDataSet[T, O]](leftSet, rightSet)
   with UnfinishedCoGroupOperation[T, O] {
 
@@ -173,11 +173,13 @@ private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
 
     // We have to use this hack, for some reason classOf[Array[T]] does not work.
     // Maybe because ObjectArrayTypeInfo does not accept the Scala Array as an array class.
-    val leftArrayType = ObjectArrayTypeInfo.getInfoFor(new Array[T](0).getClass, leftSet.getType)
-    val rightArrayType = ObjectArrayTypeInfo.getInfoFor(new Array[O](0).getClass, rightSet.getType)
+    val leftArrayType =
+      ObjectArrayTypeInfo.getInfoFor(new Array[T](0).getClass, leftSet.set.getType)
+    val rightArrayType =
+      ObjectArrayTypeInfo.getInfoFor(new Array[O](0).getClass, rightSet.set.getType)
 
     val returnType = new ScalaTupleTypeInfo[(Array[T], Array[O])](
-      classOf[(Array[T], Array[O])], Seq(leftArrayType, rightArrayType)) {
+      classOf[(Array[T], Array[O])], Seq(leftArrayType, rightArrayType), Array("_1", "_2")) {
 
       override def createSerializer: TypeSerializer[(Array[T], Array[O])] = {
         val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
@@ -195,10 +197,10 @@ private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
       }
     }
     val coGroupOperator = new CoGroupOperator[T, O, (Array[T], Array[O])](
-      leftSet, rightSet, leftKey, rightKey, coGrouper, returnType)
+      leftSet.set, rightSet.set, leftKey, rightKey, coGrouper, returnType)
 
     // sanity check solution set key mismatches
-    leftSet match {
+    leftSet.set match {
       case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
         leftKey match {
           case keyFields: Keys.FieldPositionKeys[_] =>
@@ -211,7 +213,7 @@ private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
         }
       case _ =>
     }
-    rightSet match {
+    rightSet.set match {
       case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
         rightKey match {
           case keyFields: Keys.FieldPositionKeys[_] =>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index 248c396..6ad1f74 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -68,8 +68,11 @@ private[flink] trait TypeInformationGen[C <: Context] {
     }
     val fieldsExpr = c.Expr[Seq[TypeInformation[_]]](mkList(fields))
     val instance = mkCreateTupleInstance[T](desc)(c.WeakTypeTag(desc.tpe))
+
+    val fieldNames = desc.getters map { f => Literal(Constant(f.getter.name.toString)) } toList
+    val fieldNamesExpr = c.Expr[Seq[String]](mkSeq(fieldNames))
     reify {
-      new ScalaTupleTypeInfo[T](tpeClazz.splice, fieldsExpr.splice) {
+      new ScalaTupleTypeInfo[T](tpeClazz.splice, fieldsExpr.splice, fieldNamesExpr.splice) {
         override def createSerializer: TypeSerializer[T] = {
           val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
           for (i <- 0 until getArity) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
index 5218745..2db2ff6 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
@@ -110,7 +110,7 @@ private[flink] object CrossDataSetImpl {
       }
     }
     val returnType = new ScalaTupleTypeInfo[(T, O)](
-      classOf[(T, O)], Seq(leftSet.getType, rightSet.getType)) {
+      classOf[(T, O)], Seq(leftSet.getType, rightSet.getType), Array("_1", "_2")) {
 
       override def createSerializer: TypeSerializer[(T, O)] = {
         val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index 8d24ee1..a2b09e1 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -168,8 +168,8 @@ trait UnfinishedJoinOperation[T, O] extends UnfinishedKeyPairOperation[T, O, Joi
  * i.e. the parameters of the constructor, hidden.
  */
 private[flink] class UnfinishedJoinOperationImpl[T, O](
-    leftSet: JavaDataSet[T],
-    rightSet: JavaDataSet[O],
+    leftSet: DataSet[T],
+    rightSet: DataSet[O],
     joinHint: JoinHint)
   extends UnfinishedKeyPairOperation[T, O, JoinDataSet[T, O]](leftSet, rightSet)
   with UnfinishedJoinOperation[T, O] {
@@ -181,7 +181,7 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
       }
     }
     val returnType = new ScalaTupleTypeInfo[(T, O)](
-      classOf[(T, O)], Seq(leftSet.getType, rightSet.getType)) {
+      classOf[(T, O)], Seq(leftSet.set.getType, rightSet.set.getType), Array("_1", "_2")) {
 
       override def createSerializer: TypeSerializer[(T, O)] = {
         val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
@@ -197,10 +197,10 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
       }
     }
     val joinOperator = new EquiJoin[T, O, (T, O)](
-      leftSet, rightSet, leftKey, rightKey, joiner, returnType, joinHint)
+      leftSet.set, rightSet.set, leftKey, rightKey, joiner, returnType, joinHint)
 
     // sanity check solution set key mismatches
-    leftSet match {
+    leftSet.set match {
       case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
         leftKey match {
           case keyFields: Keys.FieldPositionKeys[_] =>
@@ -213,7 +213,7 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
         }
       case _ =>
     }
-    rightSet match {
+    rightSet.set match {
       case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
         rightKey match {
           case keyFields: Keys.FieldPositionKeys[_] =>
@@ -227,6 +227,6 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
       case _ =>
     }
 
-    new JoinDataSetImpl(joinOperator, leftSet, rightSet, leftKey, rightKey)
+    new JoinDataSetImpl(joinOperator, leftSet.set, rightSet.set, leftKey, rightKey)
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
index 405158f..c63c991 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -21,7 +21,7 @@ package org.apache.flink.api
 import _root_.scala.reflect.ClassTag
 import language.experimental.macros
 import org.apache.flink.types.TypeInformation
-import org.apache.flink.api.scala.typeutils.TypeUtils
+import org.apache.flink.api.scala.typeutils.{ScalaTupleTypeInfo, TypeUtils}
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
 
 package object scala {
@@ -31,4 +31,17 @@ package object scala {
 
   // We need to wrap Java DataSet because we need the scala operations
   private[flink] def wrap[R: ClassTag](set: JavaDataSet[R]) = new DataSet[R](set)
+
+  private[flink] def fieldNames2Indices(
+      typeInfo: TypeInformation[_],
+      fields: Array[String]): Array[Int] = {
+    typeInfo match {
+      case ti: ScalaTupleTypeInfo[_] =>
+        ti.getFieldIndices(fields)
+
+      case _ =>
+        throw new UnsupportedOperationException("Specifying fields by name is only" +
+          "supported on Case Classes (for now).")
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
index 069e03b..c191e81 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
@@ -18,8 +18,7 @@
 
 package org.apache.flink.api.scala.typeutils
 
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.api.java.typeutils.{TupleTypeInfo, AtomicType, TupleTypeInfoBase}
+import org.apache.flink.api.java.typeutils.{AtomicType, TupleTypeInfoBase}
 import org.apache.flink.types.TypeInformation
 import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
 
@@ -29,7 +28,8 @@ import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
  */
 abstract class ScalaTupleTypeInfo[T <: Product](
     tupleClass: Class[T],
-    fieldTypes: Seq[TypeInformation[_]])
+    fieldTypes: Seq[TypeInformation[_]],
+    val fieldNames: Seq[String])
   extends TupleTypeInfoBase[T](tupleClass, fieldTypes: _*) {
 
   def createComparator(logicalKeyFields: Array[Int], orders: Array[Boolean]): TypeComparator[T] = {
@@ -76,5 +76,14 @@ abstract class ScalaTupleTypeInfo[T <: Product](
     new ScalaTupleComparator[T](logicalKeyFields, fieldComparators, fieldSerializers)
   }
 
+  def getFieldIndices(fields: Array[String]): Array[Int] = {
+    val result = fields map { x => fieldNames.indexOf(x) }
+    if (result.contains(-1)) {
+      throw new IllegalArgumentException("Fields '" + fields.mkString(", ") + "' are not valid for" +
+        " " + tupleClass + " with fields '" + fieldNames.mkString(", ") + "'.")
+    }
+    result
+  }
+
   override def toString = "Scala " + super.toString
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
index f8cbb62..198388e 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
@@ -24,6 +24,7 @@ import org.apache.flink.api.java.{DataSet => JavaDataSet}
 import org.apache.flink.api.java.functions.KeySelector
 import org.apache.flink.api.java.operators.Keys
 import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
+import org.apache.flink.api.scala.typeutils.ScalaTupleTypeInfo
 import org.apache.flink.types.TypeInformation
 
 /**
@@ -43,27 +44,42 @@ import org.apache.flink.types.TypeInformation
  * @tparam R The type of the resulting Operation.
  */
 private[flink] abstract class UnfinishedKeyPairOperation[T, O, R](
-    private[flink] val leftSet: JavaDataSet[T],
-    private[flink] val rightSet: JavaDataSet[O]) {
+    private[flink] val leftSet: DataSet[T],
+    private[flink] val rightSet: DataSet[O]) {
 
   private[flink] def finish(leftKey: Keys[T], rightKey: Keys[O]): R
 
   /**
    * Specify the key fields for the left side of the key based operation. This returns
-   * a [[HalfUnfinishedKeyPairOperation]] on which `isEqualTo` must be called to specify the
+   * a [[HalfUnfinishedKeyPairOperation]] on which `equalTo` must be called to specify the
    * key for the right side. The result after specifying the right side key is the finished
    * operation.
    *
-   * This only works on a Tuple [[DataSet]].
+   * This only works on Tuple [[DataSet]].
    */
   def where(leftKeys: Int*) = {
-    val leftKey = new FieldPositionKeys[T](leftKeys.toArray, leftSet.getType)
+    val leftKey = new FieldPositionKeys[T](leftKeys.toArray, leftSet.set.getType)
+    new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)
+  }
+
+  /**
+   * Specify the key fields for the left side of the key based operation. This returns
+   * a [[HalfUnfinishedKeyPairOperation]] on which `equalTo` must be called to specify the
+   * key for the right side. The result after specifying the right side key is the finished
+   * operation.
+   *
+   * This only works on a CaseClass [[DataSet]].
+   */
+  def where(firstLeftField: String, otherLeftFields: String*) = {
+    val fieldIndices = fieldNames2Indices(leftSet.set.getType, firstLeftField +: otherLeftFields.toArray)
+
+    val leftKey = new FieldPositionKeys[T](fieldIndices, leftSet.set.getType)
     new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)
   }
 
   /**
    * Specify the key selector function for the left side of the key based operation. This returns
-   * a [[HalfUnfinishedKeyPairOperation]] on which `isEqualTo` must be called to specify the
+   * a [[HalfUnfinishedKeyPairOperation]] on which `equalTo` must be called to specify the
    * key for the right side. The result after specifying the right side key is the finished
    * operation.
    */
@@ -72,7 +88,7 @@ private[flink] abstract class UnfinishedKeyPairOperation[T, O, R](
     val keyExtractor = new KeySelector[T, K] {
       def getKey(in: T) = fun(in)
     }
-    val leftKey = new Keys.SelectorFunctionKeys[T, K](keyExtractor, leftSet.getType, keyType)
+    val leftKey = new Keys.SelectorFunctionKeys[T, K](keyExtractor, leftSet.set.getType, keyType)
     new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)
   }
 }
@@ -87,7 +103,7 @@ private[flink] class HalfUnfinishedKeyPairOperation[T, O, R](
    * This only works on a Tuple [[DataSet]].
    */
   def equalTo(rightKeys: Int*): R = {
-    val rightKey = new FieldPositionKeys[O](rightKeys.toArray, unfinished.rightSet.getType)
+    val rightKey = new FieldPositionKeys[O](rightKeys.toArray, unfinished.rightSet.set.getType)
     if (!leftKey.areCompatibale(rightKey)) {
       throw new InvalidProgramException("The types of the key fields do not match. Left: " +
         leftKey + " Right: " + rightKey)
@@ -96,6 +112,26 @@ private[flink] class HalfUnfinishedKeyPairOperation[T, O, R](
   }
 
   /**
+   * Specify the key fields for the right side of the key based operation. This returns
+   * the finished operation.
+   *
+   * This only works on a CaseClass [[DataSet]].
+   */
+  def equalTo(firstRightField: String, otherRightFields: String*): R = {
+    val fieldIndices = fieldNames2Indices(
+      unfinished.rightSet.set.getType,
+      firstRightField +: otherRightFields.toArray)
+
+    val rightKey = new FieldPositionKeys[O](fieldIndices, unfinished.rightSet.set.getType)
+    if (!leftKey.areCompatibale(rightKey)) {
+      throw new InvalidProgramException("The types of the key fields do not match. Left: " +
+        leftKey + " Right: " + rightKey)
+    }
+    unfinished.finish(leftKey, rightKey)
+
+  }
+
+  /**
    * Specify the key selector function for the right side of the key based operation. This returns
    * the finished operation.
    */
@@ -105,7 +141,7 @@ private[flink] class HalfUnfinishedKeyPairOperation[T, O, R](
       def getKey(in: O) = fun(in)
     }
     val rightKey =
-      new Keys.SelectorFunctionKeys[O, K](keyExtractor, unfinished.rightSet.getType, keyType)
+      new Keys.SelectorFunctionKeys[O, K](keyExtractor, unfinished.rightSet.set.getType, keyType)
     if (!leftKey.areCompatibale(rightKey)) {
       throw new InvalidProgramException("The types of the key fields do not match. Left: " +
         leftKey + " Right: " + rightKey)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
index ae9fe22..ea35138 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
@@ -66,14 +66,54 @@ class AggregateOperatorTest {
   }
 
   @Test
+  def testFieldNamesAggregate(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should work
+    try {
+      tupleDs.aggregate(Aggregations.SUM, "_2")
+    } catch {
+      case e: Exception => Assert.fail()
+    }
+
+    // should not work: invalid field
+    try {
+      tupleDs.aggregate(Aggregations.SUM, "foo")
+      Assert.fail()
+    } catch {
+      case iae: IllegalArgumentException =>
+      case e: Exception => Assert.fail()
+    }
+
+    val longDs = env.fromCollection(emptyLongData)
+
+    // should not work: not applied to tuple DataSet
+    try {
+      longDs.aggregate(Aggregations.MIN, "_1")
+      Assert.fail()
+    } catch {
+      case uoe: InvalidProgramException =>
+      case uoe: UnsupportedOperationException =>
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
   def testAggregationTypes(): Unit = {
     try {
       val env = ExecutionEnvironment.getExecutionEnvironment
 
       val tupleDs = env.fromCollection(emptyTupleData)
 
+      // should work: multiple aggregates
       tupleDs.aggregate(Aggregations.SUM, 0).aggregate(Aggregations.MIN, 4)
+
+      // should work: nested aggregates
       tupleDs.aggregate(Aggregations.MIN, 2).aggregate(Aggregations.SUM, 1)
+
+      // should not work: average on string
       try {
         tupleDs.aggregate(Aggregations.SUM, 2)
         Assert.fail()

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
index 3608a50..2b459ab 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
@@ -94,6 +94,71 @@ class CoGroupOperatorTest {
     ds1.coGroup(ds2).where(5).equalTo(0)
   }
 
+  @Test
+  def testCoGroupKeyFieldNames1(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should work
+    try {
+      ds1.coGroup(ds2).where("_1").equalTo("_1")
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyFieldNames2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, incompatible key types
+    ds1.coGroup(ds2).where("_1").equalTo("_3")
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyFieldNames3(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, incompatible number of key fields
+    ds1.coGroup(ds2).where("_1", "_2").equalTo("_3")
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testCoGroupKeyFieldNames4(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, invalid field name
+    ds1.coGroup(ds2).where("_6").equalTo("_1")
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testCoGroupKeyFieldNames5(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, invalid field name
+    ds1.coGroup(ds2).where("_1").equalTo("bar")
+  }
+
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testCoGroupKeyFieldNames6(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should not work, field position key on custom data type
+    ds1.coGroup(ds2).where("_3").equalTo("_1")
+  }
+
   @Ignore
   @Test
   def testCoGroupKeyExpressions1(): Unit =  {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
index 693cd87..6a6e7b3 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
@@ -30,7 +30,7 @@ class DistinctOperatorTest {
   private val emptyLongData = Array[Long]()
 
   @Test
-  def testDistinctByKeyFields1(): Unit = {
+  def testDistinctByKeyIndices1(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tupleDs = env.fromCollection(emptyTupleData)
 
@@ -44,7 +44,7 @@ class DistinctOperatorTest {
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testDistinctByKeyFields2(): Unit = {
+  def testDistinctByKeyIndices2(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val longDs = env.fromCollection(emptyLongData)
 
@@ -53,7 +53,7 @@ class DistinctOperatorTest {
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testDistinctByKeyFields3(): Unit = {
+  def testDistinctByKeyIndices3(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val customDs = env.fromCollection(customTypeData)
 
@@ -62,7 +62,7 @@ class DistinctOperatorTest {
   }
 
   @Test
-  def testDistinctByKeyFields4(): Unit = {
+  def testDistinctByKeyIndices4(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tupleDs = env.fromCollection(emptyTupleData)
 
@@ -71,7 +71,7 @@ class DistinctOperatorTest {
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testDistinctByKeyFields5(): Unit = {
+  def testDistinctByKeyIndices5(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val customDs = env.fromCollection(customTypeData)
 
@@ -80,7 +80,7 @@ class DistinctOperatorTest {
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testDistinctByKeyFields6(): Unit = {
+  def testDistinctByKeyIndices6(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tupleDs = env.fromCollection(emptyTupleData)
 
@@ -89,6 +89,47 @@ class DistinctOperatorTest {
   }
 
   @Test
+  def testDistinctByKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // Should work
+    try {
+      tupleDs.distinct("_1")
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testDistinctByKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val longDs = env.fromCollection(emptyLongData)
+
+    // should not work: distinct on basic type
+    longDs.distinct("_1")
+  }
+
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testDistinctByKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val customDs = env.fromCollection(customTypeData)
+
+    // should not work: field key on custom type
+    customDs.distinct("_1")
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testDistinctByKeyFields4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should not work, invalid field
+    tupleDs.distinct("foo")
+  }
+
+  @Test
   def testDistinctByKeySelector1(): Unit = {
     val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
     try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
index 841fd52..affb007 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
@@ -33,7 +33,7 @@ class GroupingTest {
   private val emptyLongData = Array[Long]()
 
   @Test
-  def testGroupByKeyFields1(): Unit = {
+  def testGroupByKeyIndices1(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tupleDs = env.fromCollection(emptyTupleData)
 
@@ -47,7 +47,7 @@ class GroupingTest {
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testGroupByKeyFields2(): Unit = {
+  def testGroupByKeyIndices2(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val longDs = env.fromCollection(emptyLongData)
 
@@ -56,7 +56,7 @@ class GroupingTest {
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testGroupByKeyFields3(): Unit = {
+  def testGroupByKeyIndices3(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val customDs = env.fromCollection(customTypeData)
 
@@ -65,7 +65,7 @@ class GroupingTest {
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testGroupByKeyFields4(): Unit = {
+  def testGroupByKeyIndices4(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tupleDs = env.fromCollection(emptyTupleData)
 
@@ -74,7 +74,7 @@ class GroupingTest {
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testGroupByKeyFields5(): Unit = {
+  def testGroupByKeyIndices5(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tupleDs = env.fromCollection(emptyTupleData)
 
@@ -82,6 +82,47 @@ class GroupingTest {
     tupleDs.groupBy(-1)
   }
 
+  @Test
+  def testGroupByKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should work
+    try {
+      tupleDs.groupBy("_1")
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testGroupByKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val longDs = env.fromCollection(emptyLongData)
+
+    // should not work, grouping on basic type
+    longDs.groupBy("_1")
+  }
+
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testGroupByKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val customDs = env.fromCollection(customTypeData)
+
+    // should not work, field key on custom type
+    customDs.groupBy("_1")
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testGroupByKeyFields4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should not work, invalid field
+    tupleDs.groupBy("foo")
+  }
+
   @Ignore
   @Test
   def testGroupByKeyExpressions1(): Unit = {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/83debdb3/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
index fff9857..4240b6c 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
@@ -31,10 +31,12 @@ class JoinOperatorTest {
   private val emptyLongData = Array[Long]()
 
   @Test
-  def testJoinKeyFields1(): Unit = {
+  def testJoinKeyIndices1(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds1 = env.fromCollection(emptyTupleData)
     val ds2 = env.fromCollection(emptyTupleData)
+
+    // should work
     try {
       ds1.join(ds2).where(0).equalTo(0)
     }
@@ -44,45 +46,121 @@ class JoinOperatorTest {
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testJoinKeyFields2(): Unit = {
+  def testJoinKeyIndices2(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds1 = env.fromCollection(emptyTupleData)
     val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, incompatible key types
     ds1.join(ds2).where(0).equalTo(2)
   }
 
   @Test(expected = classOf[InvalidProgramException])
-  def testJoinKeyFields3(): Unit = {
+  def testJoinKeyIndices3(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds1 = env.fromCollection(emptyTupleData)
     val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, non-matching number of key indices
     ds1.join(ds2).where(0, 1).equalTo(2)
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testJoinKeyFields4(): Unit = {
+  def testJoinKeyIndices4(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds1 = env.fromCollection(emptyTupleData)
     val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, index out of range
     ds1.join(ds2).where(5).equalTo(0)
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testJoinKeyFields5(): Unit = {
+  def testJoinKeyIndices5(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds1 = env.fromCollection(emptyTupleData)
     val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, negative position
     ds1.join(ds2).where(-1).equalTo(-1)
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testJoinKeyFields6(): Unit = {
+  def testJoinKeyIndices6(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds1 = env.fromCollection(emptyTupleData)
     val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, key index on custom type
     ds1.join(ds2).where(5).equalTo(0)
   }
 
+  @Test
+  def testJoinKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // should work
+    try {
+      ds1.join(ds2).where("_1").equalTo("_1")
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, incompatible field types
+    ds1.join(ds2).where("_1").equalTo("_3")
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, non-matching number of key indices
+
+    ds1.join(ds2).where("_1", "_2").equalTo("_3")
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testJoinKeyFields4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, non-existent key
+    ds1.join(ds2).where("foo").equalTo("_1")
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testJoinKeyFields5(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // should not work, non-matching number of key indices
+    ds1.join(ds2).where("_1").equalTo("bar")
+  }
+
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testJoinKeyFields6(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, field key on custom type
+    ds1.join(ds2).where("_2").equalTo("_1")
+  }
+
   @Ignore
   @Test
   def testJoinKeyExpressions1(): Unit = {


[19/60] git commit: Perform TypeExtraction outside of Java Operators

Posted by al...@apache.org.
Perform TypeExtraction outside of Java Operators

Before, FlatMapOperator, GroupReduceOperator, MapOperator, and MapPartitionOperator
performed the Type extraction themselves while the other Operators had TypeInformation
parameters. Now the are all unified, which makes it possible to use them from the
Scala API.

Also Key extraction for selector functions is moved outside of Keys.java


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

Branch: refs/heads/master
Commit: 6bbe2a05ab4937a5a989aa2ed7451c1126d9faaa
Parents: 57b8e66
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Aug 22 18:17:48 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/api/java/DataSet.java | 23 ++++++++++++++------
 .../api/java/operators/CoGroupOperator.java     |  6 +++--
 .../api/java/operators/FlatMapOperator.java     |  6 ++---
 .../api/java/operators/GroupReduceOperator.java |  9 ++++----
 .../flink/api/java/operators/JoinOperator.java  |  6 +++--
 .../apache/flink/api/java/operators/Keys.java   | 11 +++++-----
 .../flink/api/java/operators/MapOperator.java   |  6 ++---
 .../java/operators/MapPartitionOperator.java    |  6 ++---
 .../api/java/operators/SortedGrouping.java      |  6 ++++-
 .../api/java/operators/UnsortedGrouping.java    |  6 ++++-
 10 files changed, 52 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
index de86eee..f960abd 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
@@ -66,6 +66,8 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.types.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
 
 
 /**
@@ -145,7 +147,10 @@ public abstract class DataSet<T> {
 		if (FunctionUtils.isLambdaFunction(mapper)) {
 			throw new UnsupportedLambdaExpressionException();
 		}
-		return new MapOperator<T, R>(this, mapper);
+
+		TypeInformation<R> resultType = TypeExtractor.getMapReturnTypes(mapper, this.getType());
+
+		return new MapOperator<T, R>(this, resultType, mapper);
 	}
 
 
@@ -172,7 +177,8 @@ public abstract class DataSet<T> {
 		if (mapPartition == null) {
 			throw new NullPointerException("MapPartition function must not be null.");
 		}
-		return new MapPartitionOperator<T, R>(this, mapPartition);
+		TypeInformation<R> resultType = TypeExtractor.getMapPartitionReturnTypes(mapPartition, this.getType());
+		return new MapPartitionOperator<T, R>(this, resultType, mapPartition);
 	}
 	
 	/**
@@ -194,7 +200,8 @@ public abstract class DataSet<T> {
 		if (FunctionUtils.isLambdaFunction(flatMapper)) {
 			throw new UnsupportedLambdaExpressionException();
 		}
-		return new FlatMapOperator<T, R>(this, flatMapper);
+		TypeInformation<R> resultType = TypeExtractor.getFlatMapReturnTypes(flatMapper, this.getType());
+		return new FlatMapOperator<T, R>(this, resultType, flatMapper);
 	}
 	
 	/**
@@ -340,7 +347,8 @@ public abstract class DataSet<T> {
 		if (FunctionUtils.isLambdaFunction(reducer)) {
 			throw new UnsupportedLambdaExpressionException();
 		}
-		return new GroupReduceOperator<T, R>(this, reducer);
+		TypeInformation<R> resultType = TypeExtractor.getGroupReduceReturnTypes(reducer, this.getType());
+		return new GroupReduceOperator<T, R>(this, resultType, reducer);
 	}
 
 /**
@@ -400,7 +408,8 @@ public abstract class DataSet<T> {
 	 * @return A DistinctOperator that represents the distinct DataSet.
 	 */
 	public <K> DistinctOperator<T> distinct(KeySelector<T, K> keyExtractor) {
-		return new DistinctOperator<T>(this, new Keys.SelectorFunctionKeys<T, K>(keyExtractor, getType()));
+		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, type);
+		return new DistinctOperator<T>(this, new Keys.SelectorFunctionKeys<T, K>(keyExtractor, getType(), keyType));
 	}
 	
 	/**
@@ -456,9 +465,9 @@ public abstract class DataSet<T> {
 	 * @see org.apache.flink.api.java.operators.GroupReduceOperator
 	 * @see DataSet
 	 */
-
 	public <K> UnsortedGrouping<T> groupBy(KeySelector<T, K> keyExtractor) {
-		return new UnsortedGrouping<T>(this, new Keys.SelectorFunctionKeys<T, K>(keyExtractor, getType()));
+		TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, type);
+		return new UnsortedGrouping<T>(this, new Keys.SelectorFunctionKeys<T, K>(keyExtractor, getType(), keyType));
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
index 899fc09..91ed28d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
@@ -367,7 +367,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		 * @see DataSet
 		 */
 		public <K> CoGroupOperatorSetsPredicate where(KeySelector<I1, K> keyExtractor) {
-			return new CoGroupOperatorSetsPredicate(new Keys.SelectorFunctionKeys<I1, K>(keyExtractor, input1.getType()));
+			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, input1.getType());
+			return new CoGroupOperatorSetsPredicate(new Keys.SelectorFunctionKeys<I1, K>(keyExtractor, input1.getType(), keyType));
 		}
 
 		// ----------------------------------------------------------------------------------------
@@ -431,7 +432,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			 *           Call {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.common.functions.CoGroupFunction)} to finalize the CoGroup transformation.
 			 */
 			public <K> CoGroupOperatorWithoutFunction equalTo(KeySelector<I2, K> keyExtractor) {
-				return createCoGroupOperator(new Keys.SelectorFunctionKeys<I2, K>(keyExtractor, input2.getType()));
+				TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, input2.getType());
+				return createCoGroupOperator(new Keys.SelectorFunctionKeys<I2, K>(keyExtractor, input2.getType(), keyType));
 			}
 
 			/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
index 8e531d4..4d6d30a 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
@@ -22,9 +22,9 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.FlatMapOperatorBase;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
 
 import org.apache.flink.api.java.DataSet;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * This operator represents the application of a "flatMap" function on a data set, and the
@@ -38,8 +38,8 @@ public class FlatMapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, Fl
 	protected final FlatMapFunction<IN, OUT> function;
 	
 	
-	public FlatMapOperator(DataSet<IN> input, FlatMapFunction<IN, OUT> function) {
-		super(input, TypeExtractor.getFlatMapReturnTypes(function, input.getType()));
+	public FlatMapOperator(DataSet<IN> input, TypeInformation<OUT> resultType, FlatMapFunction<IN, OUT> function) {
+		super(input, resultType);
 		
 		this.function = function;
 		extractSemanticAnnotationsFromUdf(function.getClass());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
index 7b0ad4d..9de8bd3 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
@@ -32,7 +32,6 @@ import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
 import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.types.TypeInformation;
 
 import org.apache.flink.api.java.DataSet;
@@ -58,8 +57,8 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 	 * @param input The input data set to the groupReduce function.
 	 * @param function The user-defined GroupReduce function.
 	 */
-	public GroupReduceOperator(DataSet<IN> input, GroupReduceFunction<IN, OUT> function) {
-		super(input, TypeExtractor.getGroupReduceReturnTypes(function, input.getType()));
+	public GroupReduceOperator(DataSet<IN> input, TypeInformation<OUT> resultType, GroupReduceFunction<IN, OUT> function) {
+		super(input, resultType);
 		
 		this.function = function;
 		this.grouper = null;
@@ -73,8 +72,8 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 	 * @param input The grouped input to be processed group-wise by the groupReduce function.
 	 * @param function The user-defined GroupReduce function.
 	 */
-	public GroupReduceOperator(Grouping<IN> input, GroupReduceFunction<IN, OUT> function) {
-		super(input != null ? input.getDataSet() : null, TypeExtractor.getGroupReduceReturnTypes(function, input.getDataSet().getType()));
+	public GroupReduceOperator(Grouping<IN> input, TypeInformation<OUT> resultType, GroupReduceFunction<IN, OUT> function) {
+		super(input != null ? input.getDataSet() : null, resultType);
 		
 		this.function = function;
 		this.grouper = input;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index 1ca2ec9..15447d8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -757,7 +757,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		 * @see DataSet
 		 */
 		public <K> JoinOperatorSetsPredicate where(KeySelector<I1, K> keySelector) {
-			return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<I1, K>(keySelector, input1.getType()));
+			TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+			return new JoinOperatorSetsPredicate(new Keys.SelectorFunctionKeys<I1, K>(keySelector, input1.getType(), keyType));
 		}
 		
 		// ----------------------------------------------------------------------------------------
@@ -829,7 +830,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			 * @return A DefaultJoin that represents the joined DataSet.
 			 */
 			public <K> DefaultJoin<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
-				return createJoinOperator(new Keys.SelectorFunctionKeys<I2, K>(keySelector, input2.getType()));
+				TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+				return createJoinOperator(new Keys.SelectorFunctionKeys<I2, K>(keySelector, input2.getType(), keyType));
 			}
 			
 			protected DefaultJoin<I1, I2> createJoinOperator(Keys<I2> keys2) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
index 8019cc8..9ac2b2b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
@@ -126,19 +126,18 @@ public abstract class Keys<T> {
 
 		private final KeySelector<T, K> keyExtractor;
 		private final TypeInformation<K> keyType;
-		
-		public SelectorFunctionKeys(KeySelector<T, K> keyExtractor, TypeInformation<T> type) {
+
+		public SelectorFunctionKeys(KeySelector<T, K> keyExtractor, TypeInformation<T> inputType, TypeInformation<K> keyType) {
 			if (keyExtractor == null) {
 				throw new NullPointerException("Key extractor must not be null.");
 			}
-			
+
 			this.keyExtractor = keyExtractor;
-			this.keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, type);
-			
+			this.keyType = keyType;
+
 			if (!this.keyType.isKeyType()) {
 				throw new IllegalArgumentException("Invalid type of KeySelector keys");
 			}
-			
 		}
 
 		public TypeInformation<K> getKeyType() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
index eccdeec..b27201d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
@@ -22,9 +22,9 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
 
 import org.apache.flink.api.java.DataSet;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * This operator represents the application of a "map" function on a data set, and the
@@ -40,9 +40,9 @@ public class MapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, MapOpe
 	protected final MapFunction<IN, OUT> function;
 	
 	
-	public MapOperator(DataSet<IN> input, MapFunction<IN, OUT> function) {
+	public MapOperator(DataSet<IN> input, TypeInformation<OUT> resultType, MapFunction<IN, OUT> function) {
 
-		super(input, TypeExtractor.getMapReturnTypes(function, input.getType()));
+		super(input, resultType);
 		
 		this.function = function;
 		extractSemanticAnnotationsFromUdf(function.getClass());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
index caf55f9..5b999bf 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
@@ -22,8 +22,8 @@ import org.apache.flink.api.common.functions.MapPartitionFunction;
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.MapPartitionOperatorBase;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.api.java.DataSet;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * This operator represents the application of a "mapPartition" function on a data set, and the
@@ -39,8 +39,8 @@ public class MapPartitionOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 	protected final MapPartitionFunction<IN, OUT> function;
 	
 	
-	public MapPartitionOperator(DataSet<IN> input, MapPartitionFunction<IN, OUT> function) {
-		super(input, TypeExtractor.getMapPartitionReturnTypes(function, input.getType()));
+	public MapPartitionOperator(DataSet<IN> input, TypeInformation<OUT> resultType, MapPartitionFunction<IN, OUT> function) {
+		super(input, resultType);
 		
 		this.function = function;
 		extractSemanticAnnotationsFromUdf(function.getClass());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
index 767f75a..1647055 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
@@ -27,6 +27,9 @@ import org.apache.flink.api.common.operators.Order;
 
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
 
 /**
  * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.<br/>
@@ -85,7 +88,8 @@ public class SortedGrouping<T> extends Grouping<T> {
 		if (FunctionUtils.isLambdaFunction(reducer)) {
 			throw new UnsupportedLambdaExpressionException();
 		}
-		return new GroupReduceOperator<T, R>(this, reducer);
+		TypeInformation<R> resultType = TypeExtractor.getGroupReduceReturnTypes(reducer, this.getDataSet().getType());
+		return new GroupReduceOperator<T, R>(this, resultType, reducer);
 	}
 
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6bbe2a05/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
index 702f149..fdc86de 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
@@ -29,6 +29,8 @@ import org.apache.flink.api.java.functions.SelectByMaxFunction;
 import org.apache.flink.api.java.functions.SelectByMinFunction;
 import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
 
 public class UnsortedGrouping<T> extends Grouping<T> {
 
@@ -133,7 +135,9 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 		if (FunctionUtils.isLambdaFunction(reducer)) {
 			throw new UnsupportedLambdaExpressionException();
 		}
-		return new GroupReduceOperator<T, R>(this, reducer);
+		TypeInformation<R> resultType = TypeExtractor.getGroupReduceReturnTypes(reducer, this.getDataSet().getType());
+
+		return new GroupReduceOperator<T, R>(this, resultType, reducer);
 	}
 
 	/**


[17/60] git commit: Make DistinctOperator and Keys use TupleComparatorBase

Posted by al...@apache.org.
Make DistinctOperator and Keys use TupleComparatorBase

Was TupleComparator before which does not work when used from the Scala API.


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

Branch: refs/heads/master
Commit: 785f2c4f8f3053a376083f66a2a2033edb9a8a8a
Parents: dee54cb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Aug 26 17:22:18 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../api/java/operators/DistinctOperator.java    |  5 +-
 .../apache/flink/api/java/operators/Keys.java   | 85 ++++++++++----------
 .../typeutils/runtime/TupleComparatorBase.java  |  4 +-
 3 files changed, 47 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/785f2c4f/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
index fd35773..7d8a28f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
 import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
 import org.apache.flink.types.TypeInformation;
 import org.apache.flink.util.Collector;
 
@@ -52,8 +53,8 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 		// if keys is null distinction is done on all tuple fields
 		if (keys == null) {
 			if (input.getType().isTupleType()) {
-				
-				TupleTypeInfo<?> tupleType = (TupleTypeInfo<?>) input.getType();
+
+				TupleTypeInfoBase<?> tupleType = (TupleTypeInfoBase<?>) input.getType();
 				int[] allFields = new int[tupleType.getArity()];
 				for(int i = 0; i < tupleType.getArity(); i++) {
 					allFields[i] = i;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/785f2c4f/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
index 9ac2b2b..630f674 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
@@ -23,8 +23,7 @@ import java.util.Arrays;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
 import org.apache.flink.types.TypeInformation;
 
 
@@ -32,46 +31,46 @@ public abstract class Keys<T> {
 
 
 	public abstract int getNumberOfKeyFields();
-	
+
 	public boolean isEmpty() {
 		return getNumberOfKeyFields() == 0;
 	}
-	
+
 	public abstract boolean areCompatibale(Keys<?> other);
-	
+
 	public abstract int[] computeLogicalKeyPositions();
-	
+
 	// --------------------------------------------------------------------------------------------
 	//  Specializations for field indexed / expression-based / extractor-based grouping
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static class FieldPositionKeys<T> extends Keys<T> {
-		
+
 		private final int[] fieldPositions;
 		private final TypeInformation<?>[] types;
-		
+
 		public FieldPositionKeys(int[] groupingFields, TypeInformation<T> type) {
 			this(groupingFields, type, false);
 		}
-		
+
 		public FieldPositionKeys(int[] groupingFields, TypeInformation<T> type, boolean allowEmpty) {
 			if (!type.isTupleType()) {
 				throw new InvalidProgramException("Specifying keys via field positions is only valid for tuple data types");
 			}
-			
+
 			if (!allowEmpty && (groupingFields == null || groupingFields.length == 0)) {
 				throw new IllegalArgumentException("The grouping fields must not be empty.");
 			}
-			
-			TupleTypeInfo<?> tupleType = (TupleTypeInfo<?>)type;
-	
-			this.fieldPositions = makeFields(groupingFields, (TupleTypeInfo<?>) type);
-			
+
+			TupleTypeInfoBase<?> tupleType = (TupleTypeInfoBase<?>)type;
+
+			this.fieldPositions = makeFields(groupingFields, (TupleTypeInfoBase<?>) type);
+
 			types = new TypeInformation[this.fieldPositions.length];
 			for(int i = 0; i < this.fieldPositions.length; i++) {
 				types[i] = tupleType.getTypeAt(this.fieldPositions[i]);
 			}
-			
+
 		}
 
 		@Override
@@ -81,10 +80,10 @@ public abstract class Keys<T> {
 
 		@Override
 		public boolean areCompatibale(Keys<?> other) {
-			
+
 			if (other instanceof FieldPositionKeys) {
 				FieldPositionKeys<?> oKey = (FieldPositionKeys<?>) other;
-				
+
 				if(oKey.types.length != this.types.length) {
 					return false;
 				}
@@ -94,14 +93,14 @@ public abstract class Keys<T> {
 					}
 				}
 				return true;
-				
+
 			} else if (other instanceof SelectorFunctionKeys) {
 				if(this.types.length != 1) {
 					return false;
 				}
-				
+
 				SelectorFunctionKeys<?, ?> sfk = (SelectorFunctionKeys<?, ?>) other;
-				
+
 				return sfk.keyType.equals(this.types[0]);
 			}
 			else {
@@ -113,15 +112,15 @@ public abstract class Keys<T> {
 		public int[] computeLogicalKeyPositions() {
 			return this.fieldPositions;
 		}
-	
+
 		@Override
 		public String toString() {
 			return Arrays.toString(fieldPositions);
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static class SelectorFunctionKeys<T, K> extends Keys<T> {
 
 		private final KeySelector<T, K> keyExtractor;
@@ -155,20 +154,20 @@ public abstract class Keys<T> {
 
 		@Override
 		public boolean areCompatibale(Keys<?> other) {
-			
+
 			if (other instanceof SelectorFunctionKeys) {
 				@SuppressWarnings("unchecked")
 				SelectorFunctionKeys<?, K> sfk = (SelectorFunctionKeys<?, K>) other;
-				
+
 				return sfk.keyType.equals(this.keyType);
 			}
 			else if (other instanceof FieldPositionKeys) {
 				FieldPositionKeys<?> fpk = (FieldPositionKeys<?>) other;
-						
+
 				if(fpk.types.length != 1) {
 					return false;
 				}
-				
+
 				return fpk.types[0].equals(this.keyType);
 			}
 			else {
@@ -180,15 +179,15 @@ public abstract class Keys<T> {
 		public int[] computeLogicalKeyPositions() {
 			return new int[] {0};
 		}
-		
+
 		@Override
 		public String toString() {
 			return keyExtractor + " (" + keyType + ")";
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static class ExpressionKeys<T> extends Keys<T> {
 
 		private int[] logicalPositions;
@@ -249,15 +248,15 @@ public abstract class Keys<T> {
 			return logicalPositions;
 		}
 	}
-	
-	
+
+
 	// --------------------------------------------------------------------------------------------
 	//  Utilities
 	// --------------------------------------------------------------------------------------------
-	
-	private static int[] makeFields(int[] fields, TupleTypeInfo<?> type) {
+
+	private static int[] makeFields(int[] fields, TupleTypeInfoBase<?> type) {
 		int inLength = type.getArity();
-		
+
 		// null parameter means all fields are considered
 		if (fields == null || fields.length == 0) {
 			fields = new int[inLength];
@@ -269,30 +268,30 @@ public abstract class Keys<T> {
 			return rangeCheckAndOrderFields(fields, inLength-1);
 		}
 	}
-	
+
 	private static final int[] rangeCheckAndOrderFields(int[] fields, int maxAllowedField) {
 		// order
 		Arrays.sort(fields);
-		
+
 		// range check and duplicate eliminate
 		int i = 1, k = 0;
 		int last = fields[0];
-		
+
 		if (last < 0 || last > maxAllowedField) {
 			throw new IllegalArgumentException("Tuple position is out of range.");
 		}
-		
+
 		for (; i < fields.length; i++) {
 			if (fields[i] < 0 || i > maxAllowedField) {
 				throw new IllegalArgumentException("Tuple position is out of range.");
 			}
-			
+
 			if (fields[i] != last) {
 				k++;
 				fields[k] = fields[i];
 			}
 		}
-		
+
 		// check if we eliminated something
 		if (k == fields.length - 1) {
 			return fields;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/785f2c4f/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
index cea9879..7377d72 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
@@ -246,7 +246,7 @@ public abstract class TupleComparatorBase<T> extends TypeComparator<T> implement
 	// --------------------------------------------------------------------------------------------
 	
 	@SuppressWarnings("unchecked")
-	private final void instantiateDeserializationUtils() {
+	protected final void instantiateDeserializationUtils() {
 		if (this.serializers == null) {
 			this.serializers = new TypeSerializer[this.serializerFactories.length];
 			for (int i = 0; i < this.serializers.length; i++) {
@@ -273,7 +273,7 @@ public abstract class TupleComparatorBase<T> extends TypeComparator<T> implement
 	 * @see: http://en.wikipedia.org/wiki/List_of_prime_numbers
 	 * @see: http://oeis.org/A068652
 	 */
-	protected static final int[] HASH_SALT = new int[] {
+	public static final int[] HASH_SALT = new int[] {
 		73   , 79   , 97   , 113  , 131  , 197  , 199  , 311   , 
 		337  , 373  , 719  , 733  , 919  , 971  , 991  , 1193  , 
 		1931 , 3119 , 3779 , 7793 , 7937 , 9311 , 9377 , 11939 , 


[12/60] Move RichFunctions to api.common package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
index 5e6be50..b355918 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
@@ -23,9 +23,9 @@ import java.util.List;
 
 import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
 import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
index f0229cb..21897d8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
@@ -29,7 +29,7 @@ import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
index 7c264b6..215ef2e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
@@ -24,7 +24,7 @@ import java.util.Collection;
 import java.util.LinkedList;
 
 import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.api.java.functions.RichCrossFunction;
+import org.apache.flink.api.common.functions.RichCrossFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
index 6e5cd9b..f28254d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
@@ -25,7 +25,7 @@ import java.util.LinkedList;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
index 6613bc1..d4850b2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
@@ -24,7 +24,7 @@ import java.util.Collection;
 import java.util.LinkedList;
 
 import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
index a6dd377..60a8df8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
@@ -24,7 +24,7 @@ import java.util.Collection;
 import java.util.LinkedList;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
index 2e00d32..7895690 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
@@ -27,8 +27,8 @@ import java.util.LinkedList;
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.operators.Order;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
index 290b1c2..fbac13a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
@@ -26,7 +26,7 @@ import java.util.LinkedList;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
index 4f1fb1a..75f9721 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
@@ -26,7 +26,7 @@ import java.util.LinkedList;
 import org.junit.Assert;
 
 import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
index fd7fc9f..f9279bc 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
index a636ba4..420fb9d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.LinkedList;
 
-import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
index 5390574..dbd2f06 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
@@ -25,8 +25,8 @@ import java.util.Collection;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.Program;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;


[44/60] git commit: [doc] Unify Examples Page

Posted by al...@apache.org.
[doc] Unify Examples Page

Now contains Java and Scala examples with tabs to switch between the
two.


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

Branch: refs/heads/master
Commit: 7084fa26fe8067d56fb45ad758b7879d66e50efb
Parents: 97d630d
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 19 11:53:29 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 docs/_layouts/docs.html    |   3 +-
 docs/examples.md           | 464 ++++++++++++++++++++++++++++++++++++++++
 docs/java_api_examples.md  | 307 --------------------------
 docs/scala_api_examples.md | 195 -----------------
 4 files changed, 465 insertions(+), 504 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7084fa26/docs/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/docs.html b/docs/_layouts/docs.html
index 2cf37ce..f85e75a 100644
--- a/docs/_layouts/docs.html
+++ b/docs/_layouts/docs.html
@@ -57,8 +57,7 @@
 
                     <li>Examples
                         <ul>
-                            <li><a href="java_api_examples.html">Java API</a></li>
-                            <li><a href="scala_api_examples.html">Scala API</a></li>
+                            <li><a href="examples.html">Bundled Examples</a></li>
                             <li><a href="example_connectors.html">Connecting to other systems</a></li>
                         </ul>
                     </li>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7084fa26/docs/examples.md
----------------------------------------------------------------------
diff --git a/docs/examples.md b/docs/examples.md
new file mode 100644
index 0000000..86f6fe0
--- /dev/null
+++ b/docs/examples.md
@@ -0,0 +1,464 @@
+---
+title:  "Bundled Examples"
+---
+
+* This will be replaced by the TOC
+{:toc}
+
+The following example programs showcase different applications of Flink 
+from simple word counting to graph algorithms. The code samples illustrate the 
+use of [Flink's API](programming_guide.html). 
+
+The full source code of the following and more examples can be found in the __flink-java-examples__
+or __flink-scala-examples__ module.
+
+## Word Count
+WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
+// get input data
+DataSet<String> text = getTextDataSet(env);
+
+DataSet<Tuple2<String, Integer>> counts = 
+        // split up the lines in pairs (2-tuples) containing: (word,1)
+        text.flatMap(new Tokenizer())
+        // group by the tuple field "0" and sum up tuple field "1"
+        .groupBy(0)
+        .aggregate(Aggregations.SUM, 1);
+
+counts.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+public static final class Tokenizer extends FlatMapFunction<String, Tuple2<String, Integer>> {
+
+    @Override
+    public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+        // normalize and split the line
+        String[] tokens = value.toLowerCase().split("\\W+");
+        
+        // emit the pairs
+        for (String token : tokens) {
+            if (token.length() > 0) {
+                out.collect(new Tuple2<String, Integer>(token, 1));
+            }   
+        }
+    }
+}
+~~~
+
+The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java  "WordCount example" %} implements the above described algorithm with input parameters: `<text input path>, <output path>`. As test data, any text file will do.
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+// get input data
+val text = getTextDataSet(env)
+
+val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
+  .map { (_, 1) }
+  .groupBy(0)
+  .sum(1)
+
+counts.writeAsCsv(outputPath, "\n", " ")
+~~~
+
+The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala  "WordCount example" %} implements the above described algorithm with input parameters: `<text input path>, <output path>`. As test data, any text file will do.
+
+
+</div>
+</div>
+
+## Page Rank
+
+The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
+
+In this simple example, PageRank is implemented with a [bulk iteration](java_api_guide.html#iterations) and a fixed number of iterations.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
+// get input data
+DataSet<Tuple2<Long, Double>> pagesWithRanks = getPagesWithRanksDataSet(env);
+DataSet<Tuple2<Long, Long[]>> pageLinkLists = getLinksDataSet(env);
+
+// set iterative data set
+IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(maxIterations);
+
+DataSet<Tuple2<Long, Double>> newRanks = iteration
+        // join pages with outgoing edges and distribute rank
+        .join(pageLinkLists).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch())
+        // collect and sum ranks
+        .groupBy(0).aggregate(SUM, 1)
+        // apply dampening factor
+        .map(new Dampener(DAMPENING_FACTOR, numPages));
+
+DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
+        newRanks, 
+        newRanks.join(iteration).where(0).equalTo(0)
+        // termination condition
+        .filter(new EpsilonFilter()));
+
+finalPageRanks.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+
+public static final class JoinVertexWithEdgesMatch 
+                    extends FlatMapFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>>, 
+                                            Tuple2<Long, Double>> {
+
+    @Override
+    public void flatMap(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>> value, 
+                        Collector<Tuple2<Long, Double>> out) {
+        Long[] neigbors = value.f1.f1;
+        double rank = value.f0.f1;
+        double rankToDistribute = rank / ((double) neigbors.length);
+            
+        for (int i = 0; i < neigbors.length; i++) {
+            out.collect(new Tuple2<Long, Double>(neigbors[i], rankToDistribute));
+        }
+    }
+}
+
+public static final class Dampener extends MapFunction<Tuple2<Long,Double>, Tuple2<Long,Double>> {
+    private final double dampening, randomJump;
+
+    public Dampener(double dampening, double numVertices) {
+        this.dampening = dampening;
+        this.randomJump = (1 - dampening) / numVertices;
+    }
+
+    @Override
+    public Tuple2<Long, Double> map(Tuple2<Long, Double> value) {
+        value.f1 = (value.f1 * dampening) + randomJump;
+        return value;
+    }
+}
+
+public static final class EpsilonFilter 
+                    extends FilterFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>>> {
+
+    @Override
+    public boolean filter(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>> value) {
+        return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
+    }
+}
+~~~
+
+The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java "PageRank program" %} implements the above example.
+It requires the following parameters to run: `<pages input path>, <links input path>, <output path>, <num pages>, <num iterations>`.
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+// set up execution environment
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+// read input data
+val pages = getPagesDataSet(env)
+val links = getLinksDataSet(env)
+
+// assign initial ranks to pages
+val pagesWithRanks = pages.map(p => Page(p, 1.0 / numPages))
+
+// build adjacency list from link input
+val adjacencyLists = links
+  // initialize lists
+  .map(e => AdjacencyList(e.sourceId, Array(e.targetId)))
+  // concatenate lists
+  .groupBy("sourceId").reduce {
+  (l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds)
+  }
+
+// start iteration
+val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) {
+  currentRanks =>
+    val newRanks = currentRanks
+      // distribute ranks to target pages
+      .join(adjacencyLists).where("pageId").equalTo("sourceId") {
+        (page, adjacent, out: Collector[Page]) =>
+        for (targetId <- adjacent.targetIds) {
+          out.collect(Page(targetId, page.rank / adjacent.targetIds.length))
+        }
+      }
+      // collect ranks and sum them up
+      .groupBy("pageId").aggregate(SUM, "rank")
+      // apply dampening factor
+      .map { p =>
+        Page(p.pageId, (p.rank * DAMPENING_FACTOR) + ((1 - DAMPENING_FACTOR) / numPages))
+      }
+
+    // terminate if no rank update was significant
+    val termination = currentRanks.join(newRanks).where("pageId").equalTo("pageId") {
+      (current, next, out: Collector[Int]) =>
+        // check for significant update
+        if (math.abs(current.rank - next.rank) > EPSILON) out.collect(1)
+    }
+
+    (newRanks, termination)
+}
+
+val result = finalRanks
+
+// emit result
+result.writeAsCsv(outputPath, "\n", " ")
+
+// User-defined types
+case class Link(sourceId: Long, targetId: Long)
+case class Page(pageId: Long, rank: Double)
+case class AdjacencyList(sourceId: Long, targetIds: Array[Long])
+~~~
+
+he {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala "PageRank program" %} implements the above example.
+It requires the following parameters to run: `<pages input path>, <links input path>, <output path>, <num pages>, <num iterations>`.
+</div>
+</div>
+
+Input files are plain text files and must be formatted as follows:
+- Pages represented as an (long) ID separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
+- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
+
+For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+
+## Connected Components
+
+The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
+
+This implementation uses a [delta iteration](iterations.html): Vertices that have not changed their component ID do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
+// read vertex and edge data
+DataSet<Long> vertices = getVertexDataSet(env);
+DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
+
+// assign the initial component IDs (equal to the vertex ID)
+DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
+        
+// open a delta iteration
+DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
+        verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0);
+
+// apply the step logic: 
+DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset()
+        // join with the edges
+        .join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
+        // select the minimum neighbor component ID
+        .groupBy(0).aggregate(Aggregations.MIN, 1)
+        // update if the component ID of the candidate is smaller
+        .join(iteration.getSolutionSet()).where(0).equalTo(0)
+        .flatMap(new ComponentIdFilter());
+
+// close the delta iteration (delta and new workset are identical)
+DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
+
+// emit result
+result.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+
+public static final class DuplicateValue<T> extends MapFunction<T, Tuple2<T, T>> {
+    
+    @Override
+    public Tuple2<T, T> map(T vertex) {
+        return new Tuple2<T, T>(vertex, vertex);
+    }
+}
+
+public static final class UndirectEdge 
+                    extends FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+    Tuple2<Long, Long> invertedEdge = new Tuple2<Long, Long>();
+    
+    @Override
+    public void flatMap(Tuple2<Long, Long> edge, Collector<Tuple2<Long, Long>> out) {
+        invertedEdge.f0 = edge.f1;
+        invertedEdge.f1 = edge.f0;
+        out.collect(edge);
+        out.collect(invertedEdge);
+    }
+}
+
+public static final class NeighborWithComponentIDJoin 
+                    extends JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+    @Override
+    public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithComponent, Tuple2<Long, Long> edge) {
+        return new Tuple2<Long, Long>(edge.f1, vertexWithComponent.f1);
+    }
+}
+
+public static final class ComponentIdFilter 
+                    extends FlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, 
+                                            Tuple2<Long, Long>> {
+
+    @Override
+    public void flatMap(Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> value, 
+                        Collector<Tuple2<Long, Long>> out) {
+        if (value.f0.f1 < value.f1.f1) {
+            out.collect(value.f0);
+        }
+    }
+}
+~~~
+
+The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java "ConnectedComponents program" %} implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+// set up execution environment
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+// read vertex and edge data
+// assign the initial components (equal to the vertex id)
+val vertices = getVerticesDataSet(env).map { id => (id, id) }
+
+// undirected edges by emitting for each input edge the input edges itself and an inverted
+// version
+val edges = getEdgesDataSet(env).flatMap { edge => Seq(edge, (edge._2, edge._1)) }
+
+// open a delta iteration
+val verticesWithComponents = vertices.iterateDelta(vertices, maxIterations, Array(0)) {
+  (s, ws) =>
+
+    // apply the step logic: join with the edges
+    val allNeighbors = ws.join(edges).where(0).equalTo(0) { (vertex, edge) =>
+      (edge._2, vertex._2)
+    }
+
+    // select the minimum neighbor
+    val minNeighbors = allNeighbors.groupBy(0).min(1)
+
+    // update if the component of the candidate is smaller
+    val updatedComponents = minNeighbors.join(s).where(0).equalTo(0) {
+      (newVertex, oldVertex, out: Collector[(Long, Long)]) =>
+        if (newVertex._2 < oldVertex._2) out.collect(newVertex)
+    }
+
+    // delta and new workset are identical
+    (updatedComponents, updatedComponents)
+}
+
+verticesWithComponents.writeAsCsv(outputPath, "\n", " ")
+    
+~~~
+
+The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala "ConnectedComponents program" %} implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
+</div>
+</div>
+
+Input files are plain text files and must be formatted as follows:
+- Vertices represented as IDs and separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
+- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+
+## Relational Query
+
+The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
+
+The example implements the following SQL query.
+
+~~~sql
+SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+    FROM orders, lineitem
+WHERE l_orderkey = o_orderkey
+    AND o_orderstatus = "F" 
+    AND YEAR(o_orderdate) > 1993
+    AND o_orderpriority LIKE "5%"
+GROUP BY l_orderkey, o_shippriority;
+~~~
+
+The Flink program, which implements the above query looks as follows.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
+// get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
+DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
+// get lineitem data set: (orderkey, extendedprice)
+DataSet<Tuple2<Integer, Double>> lineitems = getLineitemDataSet(env);
+
+// orders filtered by year: (orderkey, custkey)
+DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
+        // filter orders
+        orders.filter(
+            new FilterFunction<Tuple5<Integer, String, String, String, Integer>>() {
+                @Override
+                public boolean filter(Tuple5<Integer, String, String, String, Integer> t) {
+                    // status filter
+                    if(!t.f1.equals(STATUS_FILTER)) {
+                        return false;
+                    // year filter
+                    } else if(Integer.parseInt(t.f2.substring(0, 4)) <= YEAR_FILTER) {
+                        return false;
+                    // order priority filter
+                    } else if(!t.f3.startsWith(OPRIO_FILTER)) {
+                        return false;
+                    }
+                    return true;
+                }
+            })
+        // project fields out that are no longer required
+        .project(0,4).types(Integer.class, Integer.class);
+
+// join orders with lineitems: (orderkey, shippriority, extendedprice)
+DataSet<Tuple3<Integer, Integer, Double>> lineitemsOfOrders = 
+        ordersFilteredByYear.joinWithHuge(lineitems)
+                            .where(0).equalTo(0)
+                            .projectFirst(0,1).projectSecond(1)
+                            .types(Integer.class, Integer.class, Double.class);
+
+// extendedprice sums: (orderkey, shippriority, sum(extendedprice))
+DataSet<Tuple3<Integer, Integer, Double>> priceSums = 
+        // group by order and sum extendedprice
+        lineitemsOfOrders.groupBy(0,1).aggregate(Aggregations.SUM, 2);
+
+// emit result
+priceSums.writeAsCsv(outputPath);
+~~~
+
+The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java "Relational Query program" %} implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>`.
+
+</div>
+<div data-lang="scala" markdown="1">
+Coming soon...
+
+The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala "Relational Query program" %} implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>`.
+
+</div>
+</div>
+
+The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
+Take the following steps to generate arbitrary large input files for the provided Flink programs:
+
+1.  Download and unpack DBGEN
+2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
+
+~~~bash
+DATABASE = DB2
+MACHINE  = LINUX
+WORKLOAD = TPCH
+CC       = gcc
+~~~
+
+1.  Build DBGEN using *make*
+2.  Generate lineitem and orders relations using dbgen. A scale factor
+    (-s) of 1 results in a generated data set with about 1 GB size.
+
+~~~bash
+./dbgen -T o -s 1
+~~~
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7084fa26/docs/java_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/java_api_examples.md b/docs/java_api_examples.md
deleted file mode 100644
index a45b39e..0000000
--- a/docs/java_api_examples.md
+++ /dev/null
@@ -1,307 +0,0 @@
----
-title:  "Java API Examples"
----
-
-* This will be replaced by the TOC
-{:toc}
-
-The following example programs showcase different applications of Flink 
-from simple word counting to graph algorithms. The code samples illustrate the 
-use of [Flink's Java API](java_api_guide.html). 
-
-The full source code of the following and more examples can be found in the __flink-java-examples__ module.
-
-## Word Count
-WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
-
-~~~java
-// get input data
-DataSet<String> text = getTextDataSet(env);
-
-DataSet<Tuple2<String, Integer>> counts = 
-        // split up the lines in pairs (2-tuples) containing: (word,1)
-        text.flatMap(new Tokenizer())
-        // group by the tuple field "0" and sum up tuple field "1"
-        .groupBy(0)
-        .aggregate(Aggregations.SUM, 1);
-
-counts.writeAsCsv(outputPath, "\n", " ");
-
-// User-defined functions
-public static final class Tokenizer extends FlatMapFunction<String, Tuple2<String, Integer>> {
-
-    @Override
-    public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
-        // normalize and split the line
-        String[] tokens = value.toLowerCase().split("\\W+");
-        
-        // emit the pairs
-        for (String token : tokens) {
-            if (token.length() > 0) {
-                out.collect(new Tuple2<String, Integer>(token, 1));
-            }   
-        }
-    }
-}
-~~~
-
-The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java  "WordCount example" %} implements the above described algorithm with input parameters: `<text input path>, <output path>`. As test data, any text file will do.
-
-## Page Rank
-
-The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
-
-In this simple example, PageRank is implemented with a [bulk iteration](java_api_guide.html#iterations) and a fixed number of iterations.
-
-~~~java
-// get input data
-DataSet<Tuple2<Long, Double>> pagesWithRanks = getPagesWithRanksDataSet(env);
-DataSet<Tuple2<Long, Long[]>> pageLinkLists = getLinksDataSet(env);
-
-// set iterative data set
-IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(maxIterations);
-
-DataSet<Tuple2<Long, Double>> newRanks = iteration
-        // join pages with outgoing edges and distribute rank
-        .join(pageLinkLists).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch())
-        // collect and sum ranks
-        .groupBy(0).aggregate(SUM, 1)
-        // apply dampening factor
-        .map(new Dampener(DAMPENING_FACTOR, numPages));
-
-DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
-        newRanks, 
-        newRanks.join(iteration).where(0).equalTo(0)
-        // termination condition
-        .filter(new EpsilonFilter()));
-
-finalPageRanks.writeAsCsv(outputPath, "\n", " ");
-
-// User-defined functions
-
-public static final class JoinVertexWithEdgesMatch 
-                    extends FlatMapFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>>, 
-                                            Tuple2<Long, Double>> {
-
-    @Override
-    public void flatMap(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>> value, 
-                        Collector<Tuple2<Long, Double>> out) {
-        Long[] neigbors = value.f1.f1;
-        double rank = value.f0.f1;
-        double rankToDistribute = rank / ((double) neigbors.length);
-            
-        for (int i = 0; i < neigbors.length; i++) {
-            out.collect(new Tuple2<Long, Double>(neigbors[i], rankToDistribute));
-        }
-    }
-}
-
-public static final class Dampener extends MapFunction<Tuple2<Long,Double>, Tuple2<Long,Double>> {
-    private final double dampening, randomJump;
-
-    public Dampener(double dampening, double numVertices) {
-        this.dampening = dampening;
-        this.randomJump = (1 - dampening) / numVertices;
-    }
-
-    @Override
-    public Tuple2<Long, Double> map(Tuple2<Long, Double> value) {
-        value.f1 = (value.f1 * dampening) + randomJump;
-        return value;
-    }
-}
-
-public static final class EpsilonFilter 
-                    extends FilterFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>>> {
-
-    @Override
-    public boolean filter(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>> value) {
-        return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
-    }
-}
-~~~
-
-The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java "PageRank program" %} implements the above example.
-It requires the following parameters to run: `<pages input path>, <links input path>, <output path>, <num pages>, <num iterations>`.
-
-Input files are plain text files and must be formatted as follows:
-- Pages represented as an (long) ID separated by new-line characters.
-    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
-- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
-    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
-
-For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
-
-## Connected Components
-
-The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
-
-This implementation uses a [delta iteration](iterations.html): Vertices that have not changed their component ID do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
-
-~~~java
-// read vertex and edge data
-DataSet<Long> vertices = getVertexDataSet(env);
-DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
-
-// assign the initial component IDs (equal to the vertex ID)
-DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
-        
-// open a delta iteration
-DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
-        verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0);
-
-// apply the step logic: 
-DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset()
-        // join with the edges
-        .join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
-        // select the minimum neighbor component ID
-        .groupBy(0).aggregate(Aggregations.MIN, 1)
-        // update if the component ID of the candidate is smaller
-        .join(iteration.getSolutionSet()).where(0).equalTo(0)
-        .flatMap(new ComponentIdFilter());
-
-// close the delta iteration (delta and new workset are identical)
-DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
-
-// emit result
-result.writeAsCsv(outputPath, "\n", " ");
-
-// User-defined functions
-
-public static final class DuplicateValue<T> extends MapFunction<T, Tuple2<T, T>> {
-    
-    @Override
-    public Tuple2<T, T> map(T vertex) {
-        return new Tuple2<T, T>(vertex, vertex);
-    }
-}
-
-public static final class UndirectEdge 
-                    extends FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-    Tuple2<Long, Long> invertedEdge = new Tuple2<Long, Long>();
-    
-    @Override
-    public void flatMap(Tuple2<Long, Long> edge, Collector<Tuple2<Long, Long>> out) {
-        invertedEdge.f0 = edge.f1;
-        invertedEdge.f1 = edge.f0;
-        out.collect(edge);
-        out.collect(invertedEdge);
-    }
-}
-
-public static final class NeighborWithComponentIDJoin 
-                    extends JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-    @Override
-    public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithComponent, Tuple2<Long, Long> edge) {
-        return new Tuple2<Long, Long>(edge.f1, vertexWithComponent.f1);
-    }
-}
-
-public static final class ComponentIdFilter 
-                    extends FlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, 
-                                            Tuple2<Long, Long>> {
-
-    @Override
-    public void flatMap(Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> value, 
-                        Collector<Tuple2<Long, Long>> out) {
-        if (value.f0.f1 < value.f1.f1) {
-            out.collect(value.f0);
-        }
-    }
-}
-~~~
-
-The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java "ConnectedComponents program" %} implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
-
-Input files are plain text files and must be formatted as follows:
-- Vertices represented as IDs and separated by new-line characters.
-    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
-- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
-    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
-
-## Relational Query
-
-The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
-
-The example implements the following SQL query.
-
-~~~sql
-SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
-    FROM orders, lineitem
-WHERE l_orderkey = o_orderkey
-    AND o_orderstatus = "F" 
-    AND YEAR(o_orderdate) > 1993
-    AND o_orderpriority LIKE "5%"
-GROUP BY l_orderkey, o_shippriority;
-~~~
-
-The Flink Java program, which implements the above query looks as follows.
-
-~~~java
-// get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
-DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
-// get lineitem data set: (orderkey, extendedprice)
-DataSet<Tuple2<Integer, Double>> lineitems = getLineitemDataSet(env);
-
-// orders filtered by year: (orderkey, custkey)
-DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
-        // filter orders
-        orders.filter(
-            new FilterFunction<Tuple5<Integer, String, String, String, Integer>>() {
-                @Override
-                public boolean filter(Tuple5<Integer, String, String, String, Integer> t) {
-                    // status filter
-                    if(!t.f1.equals(STATUS_FILTER)) {
-                        return false;
-                    // year filter
-                    } else if(Integer.parseInt(t.f2.substring(0, 4)) <= YEAR_FILTER) {
-                        return false;
-                    // order priority filter
-                    } else if(!t.f3.startsWith(OPRIO_FILTER)) {
-                        return false;
-                    }
-                    return true;
-                }
-            })
-        // project fields out that are no longer required
-        .project(0,4).types(Integer.class, Integer.class);
-
-// join orders with lineitems: (orderkey, shippriority, extendedprice)
-DataSet<Tuple3<Integer, Integer, Double>> lineitemsOfOrders = 
-        ordersFilteredByYear.joinWithHuge(lineitems)
-                            .where(0).equalTo(0)
-                            .projectFirst(0,1).projectSecond(1)
-                            .types(Integer.class, Integer.class, Double.class);
-
-// extendedprice sums: (orderkey, shippriority, sum(extendedprice))
-DataSet<Tuple3<Integer, Integer, Double>> priceSums = 
-        // group by order and sum extendedprice
-        lineitemsOfOrders.groupBy(0,1).aggregate(Aggregations.SUM, 2);
-
-// emit result
-priceSums.writeAsCsv(outputPath);
-~~~
-
-The {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java "Relational Query program" %} implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>`.
-
-The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
-Take the following steps to generate arbitrary large input files for the provided Flink programs:
-
-1.  Download and unpack DBGEN
-2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
-
-~~~bash
-DATABASE = DB2
-MACHINE  = LINUX
-WORKLOAD = TPCH
-CC       = gcc
-~~~
-
-1.  Build DBGEN using *make*
-2.  Generate lineitem and orders relations using dbgen. A scale factor
-    (-s) of 1 results in a generated data set with about 1 GB size.
-
-~~~bash
-./dbgen -T o -s 1
-~~~
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7084fa26/docs/scala_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_examples.md b/docs/scala_api_examples.md
deleted file mode 100644
index b6689f0..0000000
--- a/docs/scala_api_examples.md
+++ /dev/null
@@ -1,195 +0,0 @@
----
-title:  "Scala API Examples"
----
-
-The following example programs showcase different applications of Flink from simple word counting to graph algorithms.
-The code samples illustrate the use of [Flink's Scala API](scala_api_guide.html). 
-
-The full source code of the following and more examples can be found in the [flink-scala-examples](https://github.com/apache/incubator-flink/tree/master/flink-examples/flink-scala-examples) module.
-
-# Word Count
-
-WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
-
-```scala
-// read input data
-val input = TextFile(textInput)
-
-// tokenize words
-val words = input.flatMap { _.split(" ") map { (_, 1) } }
-
-// count by word
-val counts = words.groupBy { case (word, _) => word }
-  .reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
-
-val output = counts.write(wordsOutput, CsvOutputFormat()))
-```
-
-The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala "WordCount example" %} implements the above described algorithm with input parameters: `<degree of parallelism>, <text input path>, <output path>`. As test data, any text file will do.
-
-# Page Rank
-
-The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
-
-In this simple example, PageRank is implemented with a [bulk iteration](java_api_guide.html#iterations) and a fixed number of iterations.
-
-```scala
-// cases classes so we have named fields
-case class PageWithRank(pageId: Long, rank: Double)
-case class Edge(from: Long, to: Long, transitionProbability: Double)
-
-// constants for the page rank formula
-val dampening = 0.85
-val randomJump = (1.0 - dampening) / NUM_VERTICES
-val initialRank = 1.0 / NUM_VERTICES
-  
-// read inputs
-val pages = DataSource(verticesPath, CsvInputFormat[Long]())
-val edges = DataSource(edgesPath, CsvInputFormat[Edge]())
-
-// assign initial rank
-val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
-
-// the iterative computation
-def computeRank(ranks: DataSet[PageWithRank]) = {
-
-    // send rank to neighbors
-    val ranksForNeighbors = ranks join edges
-        where { _.pageId } isEqualTo { _.from }
-        map { (p, e) => (e.to, p.rank * e.transitionProbability) }
-    
-    // gather ranks per vertex and apply page rank formula
-    ranksForNeighbors .groupBy { case (node, rank) => node }
-                      .reduce { (a, b) => (a._1, a._2 + b._2) }
-                      .map {case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
-}
-
-// invoke iteratively
-val finalRanks = pagesWithRank.iterate(numIterations, computeRank)
-val output = finalRanks.write(outputPath, CsvOutputFormat())
-```
-
-
-
-The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala "PageRank program" %} implements the above example.
-It requires the following parameters to run: `<pages input path>, <link input path>, <output path>, <num pages>, <num iterations>`.
-
-Input files are plain text files and must be formatted as follows:
-- Pages represented as an (long) ID separated by new-line characters.
-    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
-- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
-    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
-
-For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
-
-# Connected Components
-
-The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
-
-This implementation uses a [delta iteration](iterations.html): Vertices that have not changed their component id do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
-
-```scala
-// define case classes
-case class VertexWithComponent(vertex: Long, componentId: Long)
-case class Edge(from: Long, to: Long)
-
-// get input data
-val vertices = DataSource(verticesPath, CsvInputFormat[Long]())
-val directedEdges = DataSource(edgesPath, CsvInputFormat[Edge]())
-
-// assign each vertex its own ID as component ID
-val initialComponents = vertices map { v => VertexWithComponent(v, v) }
-val undirectedEdges = directedEdges flatMap { e => Seq(e, Edge(e.to, e.from)) }
-
-def propagateComponent(s: DataSet[VertexWithComponent], ws: DataSet[VertexWithComponent]) = {
-  val allNeighbors = ws join undirectedEdges
-        where { _.vertex } isEqualTo { _.from }
-        map { (v, e) => VertexWithComponent(e.to, v.componentId ) }
-    
-    val minNeighbors = allNeighbors groupBy { _.vertex } reduceGroup { cs => cs minBy { _.componentId } }
-
-    // updated solution elements == new workset
-    val s1 = s join minNeighbors
-        where { _.vertex } isEqualTo { _.vertex }
-        flatMap { (curr, candidate) =>
-            if (candidate.componentId < curr.componentId) Some(candidate) else None
-        }
-
-  (s1, s1)
-}
-
-val components = initialComponents.iterateWithDelta(initialComponents, { _.vertex }, propagateComponent,
-                    maxIterations)
-val output = components.write(componentsOutput, CsvOutputFormat())
-```
-
-The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala "ConnectedComponents program" %} implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
-
-Input files are plain text files and must be formatted as follows:
-- Vertices represented as IDs and separated by new-line characters.
-    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
-- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
-    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
-
-# Relational Query
-
-The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
-
-The example implements the following SQL query.
-
-```sql
-SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
-    FROM orders, lineitem
-WHERE l_orderkey = o_orderkey
-    AND o_orderstatus = "F" 
-    AND YEAR(o_orderdate) > 1993
-    AND o_orderpriority LIKE "5%"
-GROUP BY l_orderkey, o_shippriority;
-```
-
-The Flink Scala program, which implements the above query looks as follows.
-
-```scala
-// --- define some custom classes to address fields by name ---
-case class Order(orderId: Int, status: Char, date: String, orderPriority: String, shipPriority: Int)
-case class LineItem(orderId: Int, extendedPrice: Double)
-case class PrioritizedOrder(orderId: Int, shipPriority: Int, revenue: Double)
-
-val orders = DataSource(ordersInputPath, DelimitedInputFormat(parseOrder))
-val lineItem2600s = DataSource(lineItemsInput, DelimitedInputFormat(parseLineItem))
-
-val filteredOrders = orders filter { o => o.status == "F" && o.date.substring(0, 4).toInt > 1993 && o.orderPriority.startsWith("5") }
-
-val prioritizedItems = filteredOrders join lineItems
-    where { _.orderId } isEqualTo { _.orderId } // join on the orderIds
-    map { (o, li) => PrioritizedOrder(o.orderId, o.shipPriority, li.extendedPrice) }
-
-val prioritizedOrders = prioritizedItems
-    groupBy { pi => (pi.orderId, pi.shipPriority) } 
-    reduce { (po1, po2) => po1.copy(revenue = po1.revenue + po2.revenue) }
-
-val output = prioritizedOrders.write(ordersOutput, CsvOutputFormat(formatOutput))
-```
-
-The {% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala "Relational Query program" %} implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>, <degree of parallelism>`.
-
-The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
-Take the following steps to generate arbitrary large input files for the provided Flink programs:
-
-1.  Download and unpack DBGEN
-2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
-
-```bash
-DATABASE = DB2
-MACHINE  = LINUX
-WORKLOAD = TPCH
-CC       = gcc
-```
-
-1.  Build DBGEN using *make*
-2.  Generate lineitem and orders relations using dbgen. A scale factor
-    (-s) of 1 results in a generated data set with about 1 GB size.
-
-```bash
-./dbgen -T o -s 1
-```
\ No newline at end of file


[31/60] git commit: Transitive closure Scala example

Posted by al...@apache.org.
Transitive closure Scala example


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

Branch: refs/heads/master
Commit: e0f2440d674dc6548c49ae92a8f1fcf42cd242f0
Parents: a8dd958
Author: Kostas Tzoumas <ko...@kostass-mbp.fritz.box>
Authored: Wed Sep 10 16:20:46 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../java/graph/TransitiveClosureNaive.java      |  22 ++++
 flink-examples/flink-scala-examples/pom.xml     |  24 +++-
 .../scala/graph/TransitiveClosureNaive.scala    | 119 +++++++++++++++++++
 .../TransitiveClosureITCase.java                |  60 ++++++++++
 4 files changed, 224 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e0f2440d/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
index 281439e..0745d73 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
@@ -19,6 +19,7 @@
 
 package org.apache.flink.example.java.graph;
 
+import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.java.DataSet;
@@ -30,6 +31,9 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.example.java.graph.util.ConnectedComponentsData;
 import org.apache.flink.util.Collector;
 
+import java.util.HashSet;
+import java.util.Set;
+
 @SuppressWarnings("serial")
 public class TransitiveClosureNaive implements ProgramDescription {
 
@@ -73,6 +77,24 @@ public class TransitiveClosureNaive implements ProgramDescription {
 					}
 				});
 
+		DataSet<Tuple2<Long,Long>> newPaths = paths
+				.coGroup(nextPaths)
+				.where(0).equalTo(0)
+				.with(new CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>>() {
+					Set prevSet = new HashSet<Tuple2<Long,Long>>();
+					@Override
+					public void coGroup(Iterable<Tuple2<Long, Long>> prevPaths, Iterable<Tuple2<Long, Long>> nextPaths, Collector<Tuple2<Long, Long>> out) throws Exception {
+						for (Tuple2<Long,Long> prev : prevPaths) {
+							prevSet.add(prev);
+						}
+						for (Tuple2<Long,Long> next: nextPaths) {
+							if (!prevSet.contains(next)) {
+								out.collect(next);
+							}
+						}
+					}
+				});
+
 		DataSet<Tuple2<Long, Long>> transitiveClosure = paths.closeWith(nextPaths);
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e0f2440d/flink-examples/flink-scala-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/pom.xml b/flink-examples/flink-scala-examples/pom.xml
index a6801f8..3b50c8b 100644
--- a/flink-examples/flink-scala-examples/pom.xml
+++ b/flink-examples/flink-scala-examples/pom.xml
@@ -281,7 +281,29 @@ under the License.
 					</execution>
 					
 					-->
-		
+
+                    <execution>
+                        <id>TransitiveClosureNaive</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>jar</goal>
+                        </goals>
+
+                        <configuration>
+                            <classifier>TransitiveClosureNaive</classifier>
+
+                            <archive>
+                                <manifestEntries>
+                                    <program-class>org.apache.flink.examples.scala.graph.TransitiveClosureNaive</program-class>
+                                </manifestEntries>
+                            </archive>
+
+                            <includes>
+                                <include>**/wordcount/TransitiveClosureNaive*.class</include>
+                            </includes>
+                        </configuration>
+                    </execution>
+
 				</executions>
 			</plugin>
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e0f2440d/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
index 858ce30..86d83db 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
@@ -1,3 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.graph
+
+import org.apache.flink.api.scala._
+import org.apache.flink.example.java.graph.util.ConnectedComponentsData
+import org.apache.flink.util.Collector
+
+object TransitiveClosureNaive {
+
+
+	def main (args: Array[String]): Unit = {
+		if (!parseParameters(args)) {
+			return
+		}
+
+		val env = ExecutionEnvironment.getExecutionEnvironment
+
+		val edges = getEdgesDataSet(env)
+
+		val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long,Long)] =>
+
+			val nextPaths = prevPaths
+				.join(edges)
+				.where(1).equalTo(0) {
+					(left,right) => Some((left._1,right._2))
+				}
+				.union(prevPaths)
+				.groupBy(0,1)
+				.reduce((l,r) => l)
+
+			val terminate = prevPaths
+				.coGroup(nextPaths)
+				.where(0).equalTo(0) {
+					(prev, next, out: Collector[(Long, Long)]) => {
+						val prevPaths = prev.toList
+						for (n <- next)
+							if (!prevPaths.contains(n))
+								out.collect(n)
+					}
+			}
+			(nextPaths, terminate)
+		}
+
+		if (fileOutput)
+			paths.writeAsCsv(outputPath, "\n", " ")
+		else
+			paths.print()
+
+		env.execute("Scala Transitive Closure Example")
+
+
+	}
+
+
+	private var fileOutput: Boolean = false
+	private var edgesPath: String = null
+	private var outputPath: String = null
+	private var maxIterations: Int = 10
+
+	private def parseParameters(programArguments: Array[String]): Boolean = {
+		if (programArguments.length > 0) {
+			fileOutput = true
+			if (programArguments.length == 3) {
+				edgesPath = programArguments(0)
+				outputPath = programArguments(1)
+				maxIterations = Integer.parseInt(programArguments(2))
+			}
+			else {
+				System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of iterations>")
+				return false
+			}
+		}
+		else {
+			System.out.println("Executing TransitiveClosure example with default parameters and built-in default data.")
+			System.out.println("  Provide parameters to read input data from files.")
+			System.out.println("  See the documentation for the correct format of input files.")
+			System.out.println("  Usage: TransitiveClosure <edges path> <result path> <max number of iterations>")
+		}
+		return true
+	}
+
+	private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = {
+		if (fileOutput) {
+			env.readCsvFile[(Long, Long)](
+				edgesPath,
+				fieldDelimiter = ' ',
+				includedFields = Array(0, 1))
+				.map { x => (x._1, x._2)}
+		}
+		else {
+			val edgeData = ConnectedComponentsData.EDGES map {
+				case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
+			}
+			env.fromCollection(edgeData)
+		}
+	}
+}
+
+
+
+
+
+
 ///**
 // * Licensed to the Apache Software Foundation (ASF) under one
 // * or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e0f2440d/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java
new file mode 100644
index 0000000..1bf25a6
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.test.exampleScalaPrograms;
+
+
+import java.io.BufferedReader;
+
+import org.apache.flink.examples.scala.graph.TransitiveClosureNaive;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.testdata.TransitiveClosureData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class TransitiveClosureITCase extends JavaProgramTestBase {
+
+    private static final long SEED = 0xBADC0FFEEBEEFL;
+
+    private static final int NUM_VERTICES = 1000;
+
+    private static final int NUM_EDGES = 10000;
+
+    private String edgesPath;
+    private String resultPath;
+
+
+    @Override
+    protected void preSubmit() throws Exception {
+        edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
+        resultPath = getTempFilePath("results");
+    }
+
+    @Override
+    protected void testProgram() throws Exception {
+        TransitiveClosureNaive.main(new String [] {edgesPath, resultPath, "5"});
+    }
+
+    @Override
+    protected void postSubmit() throws Exception {
+        for (BufferedReader reader : getResultReader(resultPath)) {
+            TransitiveClosureData.checkOddEvenResult(reader);
+        }
+    }
+}
+


[38/60] git commit: Add Scala API completeness Test

Posted by al...@apache.org.
Add Scala API completeness Test

This is very naive right now, just checks whether a function of the same
name is available. There are lists and regex patterns of excluded
functions since not all functions in the Java API need to be available
in the Scala API.


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

Branch: refs/heads/master
Commit: dbc680f25d436c8bfe0d492f17f49abddcb6607f
Parents: 1a4147f
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 12 16:26:38 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/api/scala/DataSet.scala    |  15 ++
 .../api/scala/ScalaAPICompletenessTest.scala    | 160 +++++++++++++++++++
 2 files changed, 175 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dbc680f2/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 8f14c0a..d04f968 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -826,6 +826,21 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
     wrap(result)
   }
 
+  // -------------------------------------------------------------------------------------------
+  //  Custom Operators
+  // -------------------------------------------------------------------------------------------
+
+  // Keep it out until we have an actual use case for this.
+//  /**
+//   * Runs a [[CustomUnaryOperation]] on the data set. Custom operations are typically complex
+//   * operators that are composed of multiple steps.
+//   */
+//  def runOperation[R: ClassTag](operation: CustomUnaryOperation[T, R]): DataSet[R] = {
+//    Validate.notNull(operation, "The custom operator must not be null.")
+//    operation.setInput(this.set)
+//    wrap(operation.createResult)
+//  }
+
   // --------------------------------------------------------------------------------------------
   //  Union
   // --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dbc680f2/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
new file mode 100644
index 0000000..5bfa9b4
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala
+
+import java.lang.reflect.Method
+
+import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+
+import scala.language.existentials
+
+import org.junit.Assert._
+import org.junit.Test
+
+/**
+ * This checks whether the Scala API is up to feature parity with the Java API. Right now is very
+ * simple, it is only checked whether a method with the same name exists.
+ *
+ * When adding excluded methods to the lists you should give a good reason in a comment.
+ *
+ * Note: This is inspired by the JavaAPICompletenessChecker from Spark.
+ */
+class ScalaAPICompletenessTest {
+
+  private def isExcludedByName(method: Method): Boolean = {
+    val name = method.getDeclaringClass.getName + "." + method.getName
+    val excludedNames = Seq(
+      // These are only used internally. Should be internal API but Java doesn't have
+      // private[flink].
+      "org.apache.flink.api.java.DataSet.getExecutionEnvironment",
+      "org.apache.flink.api.java.DataSet.getType",
+      "org.apache.flink.api.java.operators.Operator.getResultType",
+      "org.apache.flink.api.java.operators.Operator.getName",
+      "org.apache.flink.api.java.operators.Grouping.getDataSet",
+      "org.apache.flink.api.java.operators.Grouping.getKeys",
+      "org.apache.flink.api.java.operators.SingleInputOperator.getInput",
+      "org.apache.flink.api.java.operators.SingleInputOperator.getInputType",
+      "org.apache.flink.api.java.operators.TwoInputOperator.getInput1",
+      "org.apache.flink.api.java.operators.TwoInputOperator.getInput2",
+      "org.apache.flink.api.java.operators.TwoInputOperator.getInput1Type",
+      "org.apache.flink.api.java.operators.TwoInputOperator.getInput2Type",
+
+      // This is really just a mapper, which in Scala can easily expressed as a map lambda
+      "org.apache.flink.api.java.DataSet.writeAsFormattedText",
+
+      // Exclude minBy and maxBy for now, since there is some discussion about our aggregator
+      // semantics
+      "org.apache.flink.api.java.DataSet.minBy",
+      "org.apache.flink.api.java.DataSet.maxBy",
+      "org.apache.flink.api.java.operators.UnsortedGrouping.minBy",
+      "org.apache.flink.api.java.operators.UnsortedGrouping.maxBy"
+
+    )
+    val excludedPatterns = Seq(
+      // We don't have project on tuples in the Scala API
+      """^org\.apache\.flink\.api.java.*project""",
+
+      // I don't want to have withParameters in the API since I consider Configuration to be
+      // deprecated. But maybe thats just me ...
+      """^org\.apache\.flink\.api.java.*withParameters""",
+
+      // These are only used internally. Should be internal API but Java doesn't have
+      // private[flink].
+      """^org\.apache\.flink\.api.java.*getBroadcastSets""",
+      """^org\.apache\.flink\.api.java.*setSemanticProperties""",
+      """^org\.apache\.flink\.api.java.*getSemanticProperties""",
+      """^org\.apache\.flink\.api.java.*getParameters""",
+
+      // Commented out for now until we have a use case for this.
+      """^org\.apache\.flink\.api.java.*runOperation""",
+
+      // Object methods
+      """^.*notify""",
+      """^.*wait""",
+      """^.*notifyAll""",
+      """^.*equals""",
+      """^.*toString""",
+      """^.*getClass""",
+      """^.*hashCode"""
+    ).map(_.r)
+    lazy val excludedByPattern =
+      excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).nonEmpty
+    name.contains("$") || excludedNames.contains(name) || excludedByPattern
+  }
+
+  private def isExcludedByInterface(method: Method): Boolean = {
+    val excludedInterfaces =
+      Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil")
+    def toComparisionKey(method: Method) =
+      (method.getReturnType, method.getName, method.getGenericReturnType)
+    val interfaces = method.getDeclaringClass.getInterfaces.filter { i =>
+      excludedInterfaces.contains(i.getName)
+    }
+    val excludedMethods = interfaces.flatMap(_.getMethods.map(toComparisionKey))
+    excludedMethods.contains(toComparisionKey(method))
+  }
+
+  private def checkMethods(
+      javaClassName: String,
+      scalaClassName: String,
+      javaClass: Class[_],
+      scalaClass: Class[_]) {
+    val javaMethods = javaClass.getMethods
+      .filterNot(_.isAccessible)
+      .filterNot(isExcludedByName)
+      .filterNot(isExcludedByInterface)
+      .map(m => m.getName).toSet
+
+    val scalaMethods = scalaClass.getMethods
+      .filterNot(_.isAccessible)
+      .filterNot(isExcludedByName)
+      .filterNot(isExcludedByInterface)
+      .map(m => m.getName).toSet
+
+    val missingMethods = javaMethods -- scalaMethods
+
+    for (method <- missingMethods) {
+      fail("Method " + method + " from " + javaClass + " is missing from " + scalaClassName + ".")
+    }
+  }
+
+  @Test
+  def testCompleteness(): Unit = {
+    checkMethods("DataSet", "DataSet", classOf[JavaDataSet[_]], classOf[DataSet[_]])
+
+    checkMethods("Operator", "DataSet", classOf[Operator[_, _]], classOf[DataSet[_]])
+
+    checkMethods("UnsortedGrouping", "GroupedDataSet",
+      classOf[UnsortedGrouping[_]], classOf[GroupedDataSet[_]])
+
+    checkMethods("SortedGrouping", "GroupedDataSet",
+      classOf[SortedGrouping[_]], classOf[GroupedDataSet[_]])
+
+    checkMethods("SingleInputOperator", "DataSet",
+      classOf[SingleInputOperator[_, _, _]], classOf[DataSet[_]])
+
+    checkMethods("TwoInputOperator", "DataSet", classOf[TwoInputOperator[_, _, _, _]], classOf[DataSet[_]])
+
+    checkMethods("SingleInputUdfOperator", "DataSet",
+      classOf[SingleInputUdfOperator[_, _, _]], classOf[DataSet[_]])
+
+    checkMethods("TwoInputUdfOperator", "DataSet",
+      classOf[TwoInputUdfOperator[_, _, _, _]], classOf[DataSet[_]])
+  }
+}


[58/60] git commit: [doc] Add automatic generation of Scaladoc

Posted by al...@apache.org.
[doc] Add automatic generation of Scaladoc


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

Branch: refs/heads/master
Commit: 0c79538ac5120b7f64464a50413f0ce0fd750145
Parents: 69808fd
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Sep 20 17:42:31 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 13:43:03 2014 +0200

----------------------------------------------------------------------
 docs/_includes/navbar.html    |  4 ++++
 docs/_plugins/build_apidoc.rb | 17 ++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c79538a/docs/_includes/navbar.html
----------------------------------------------------------------------
diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html
index 5c7b8ac..29b514d 100644
--- a/docs/_includes/navbar.html
+++ b/docs/_includes/navbar.html
@@ -21,6 +21,10 @@
           <a href="api/java/index.html">Javadoc</a>
         </li>
 
+        <li>
+          <a href="api/scala/index.html#org.apache.flink.api.scala.package">Scaladoc</a>
+        </li>
+
       </ul>
     </div>
   </div>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c79538a/docs/_plugins/build_apidoc.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/build_apidoc.rb b/docs/_plugins/build_apidoc.rb
index 8caa971..5f17197 100644
--- a/docs/_plugins/build_apidoc.rb
+++ b/docs/_plugins/build_apidoc.rb
@@ -19,7 +19,7 @@ require 'rubygems'
 include FileUtils
 
 if ENV['BUILD_API'] == '1' then
-  # Build Javadoc and Scaladoc
+  # Build Javadoc
 
   cd("..")
 
@@ -48,4 +48,19 @@ if ENV['BUILD_API'] == '1' then
   puts "cp -r " + source + "/. " + dest
   cp_r(source + "/.", dest)
 
+  # Build Scaladoc
+  cd("../flink-scala")
+
+  puts "Generating Scaladoc"
+  puts `mvn scala:doc`
+
+  cd("../docs")
+
+  source = "../flink-scala/target/site/scaladocs"
+  dest = "api/scala/"
+
+  puts "cp -r " + source + "/. " + dest
+  cp_r(source + "/.", dest)
+
+
 end


[57/60] git commit: [doc] Unify "DataSet Transformations" page

Posted by al...@apache.org.
[doc] Unify "DataSet Transformations" page


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

Branch: refs/heads/master
Commit: 69808fddad3ee9ab445c1dd6063c0488636546b1
Parents: c778d28
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Sep 20 08:48:53 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 13:43:03 2014 +0200

----------------------------------------------------------------------
 docs/dataset_transformations.md | 164 ++++++++++++++++++++++++-----------
 1 file changed, 111 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/69808fdd/docs/dataset_transformations.md
----------------------------------------------------------------------
diff --git a/docs/dataset_transformations.md b/docs/dataset_transformations.md
index e027eb5..fec796b 100644
--- a/docs/dataset_transformations.md
+++ b/docs/dataset_transformations.md
@@ -534,7 +534,8 @@ DataSet<Tuple3<Integer, String, Double>> output = input
 <div data-lang="scala" markdown="1">
 
 ~~~scala
-
+val input: DataSet[(Int, String, Double)] = // [...]
+val output = input.groupBy(1).aggregate(SUM, 0).and(MIN, 2)
 ~~~
 
 </div>
@@ -547,8 +548,8 @@ In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggrega
 
 ### Reduce on full DataSet
 
-The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a DataSet.
-The `ReduceFunction` subsequently combines pairs of elements into one element until only a single element remains.
+The Reduce transformation applies a user-defined reduce function to all elements of a DataSet.
+The reduce function subsequently combines pairs of elements into one element until only a single element remains.
 
 The following code shows how to sum all elements of an Integer DataSet:
 
@@ -573,18 +574,19 @@ DataSet<Integer> sum = intNumbers.reduce(new IntSummer());
 <div data-lang="scala" markdown="1">
 
 ~~~scala
-
+val intNumbers = env.fromElements(1,2,3)
+val sum = intNumbers.reduce (_ + _)
 ~~~
 
 </div>
 </div>
 
-Reducing a full DataSet using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
+Reducing a full DataSet using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a reduce function is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
 
 ### GroupReduce on full DataSet
 
-The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a DataSet.
-A `GroupReduceFunction` can iterate over all elements of DataSet and return an arbitrary number of result elements.
+The GroupReduce transformation applies a user-defined group-reduce function on all elements of a DataSet.
+A group-reduce can iterate over all elements of DataSet and return an arbitrary number of result elements.
 
 The following example shows how to apply a GroupReduce transformation on a full DataSet:
 
@@ -601,17 +603,22 @@ DataSet<Double> output = input.reduceGroup(new MyGroupReducer());
 <div data-lang="scala" markdown="1">
 
 ~~~scala
-
+val input: DataSet[Int] = // [...]
+val output = input.reduceGroup(new MyGroupReducer())
 ~~~
 
 </div>
 </div>
 
-**Note:** A GroupReduce transformation on a full DataSet cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
+**Note:** A GroupReduce transformation on a full DataSet cannot be done in parallel if the
+group-reduce function is not combinable. Therefore, this can be a very compute intensive operation.
+See the paragraph on "Combineable Group-Reduce Functions" above to learn how to implement a
+combinable group-reduce function.
 
 ### Aggregate on full Tuple DataSet
 
-There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+There are some common aggregation operations that are frequently used. The Aggregate transformation
+provides the following build-in aggregation functions:
 
 - Sum,
 - Min, and
@@ -635,6 +642,8 @@ DataSet<Tuple2<Integer, Double>> output = input
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+val input: DataSet[(Int, String, Double)] = // [...]
+val output = input.aggregate(SUM, 0).and(MIN, 2)
 
 ~~~
 
@@ -647,14 +656,15 @@ DataSet<Tuple2<Integer, Double>> output = input
 
 The Join transformation joins two DataSets into one DataSet. The elements of both DataSets are joined on one or more keys which can be specified using
 
-- a `KeySelector` function or
+- a key-selector function or
 - one or more field position keys (Tuple DataSet only).
+- Case Class Fields
 
 There are a few different ways to perform a Join transformation which are shown in the following.
 
 #### Default Join (Join into Tuple2)
 
-The default Join transformation produces a new TupleDataSet with two fields. Each tuple holds a joined element of the first input DataSet in the first tuple field and a matching element of the second input DataSet in the second field.
+The default Join transformation produces a new Tuple DataSet with two fields. Each tuple holds a joined element of the first input DataSet in the first tuple field and a matching element of the second input DataSet in the second field.
 
 The following code shows a default Join transformation using field position keys:
 
@@ -675,18 +685,20 @@ DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Double, Integer>>>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
-
+val input1: DataSet[(Int, String)] = // [...]
+val input2: DataSet[(Double, Int)] = // [...]
+val result = input1.join(input2).where(0).equalTo(1)
 ~~~
 
 </div>
 </div>
 
-#### Join with JoinFunction
+#### Join with Join-Function
 
-A Join transformation can also call a user-defined `JoinFunction` to process joining tuples.
-A `JoinFunction` receives one element of the first input DataSet and one element of the second input DataSet and returns exactly one element.
+A Join transformation can also call a user-defined join function to process joining tuples.
+A join function receives one element of the first input DataSet and one element of the second input DataSet and returns exactly one element.
 
-The following code performs a join of DataSet with custom java objects and a Tuple DataSet using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
+The following code performs a join of DataSet with custom java objects and a Tuple DataSet using key-selector functions and shows how to use a user-defined join function:
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -734,18 +746,29 @@ DataSet<Tuple2<String, Double>>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+case class Rating(name: String, category: String, points: Int)
+
+val ratings: DataSet[Ratings] = // [...]
+val weights: DataSet[(String, Double)] = // [...]
 
+val weightedRatings = ratings.join(weights).where("category").equalTo(0) {
+  (rating, weight) => (rating.name, rating.points * weight._2)
+}
 ~~~
 
 </div>
 </div>
 
-#### Join with FlatJoinFunction
+#### Join with Flat-Join Function
 
-Analogous to Map and FlatMap, a FlatJoin function behaves in the same
-way as a JoinFunction, but instead of returning one element, it can
+Analogous to Map and FlatMap, a FlatJoin behaves in the same
+way as a Join, but instead of returning one element, it can
 return (collect), zero, one, or more elements.
-{% highlight java %}
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 public class PointWeighter
          implements FlatJoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
   @Override
@@ -760,15 +783,12 @@ public class PointWeighter
 DataSet<Tuple2<String, Double>>
             weightedRatings =
             ratings.join(weights) // [...]
-{% endhighlight %}
+~~~
 
-#### Join with Projection
+#### Join with Projection (Java Only)
 
 A Join transformation can construct result tuples using a projection as shown here:
 
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-
 ~~~java
 DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
 DataSet<Tuple2<Integer, Double>> input2 = // [...]
@@ -784,19 +804,28 @@ DataSet<Tuple4<Integer, String, Double, Byte>
                   .types(Integer.class, String.class, Double.class, Byte.class);
 ~~~
 
+`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output Tuple. The order of indexes defines the order of fields in the output tuple.
+The join projection works also for non-Tuple DataSets. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output Tuple.
+
 </div>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+case class Rating(name: String, category: String, points: Int)
+
+val ratings: DataSet[Ratings] = // [...]
+val weights: DataSet[(String, Double)] = // [...]
+
+val weightedRatings = ratings.join(weights).where("category").equalTo(0) {
+  (rating, weight, out: Collector[(String, Double)] =>
+    if (weight._2 > 0.1) out.collect(left.name, left.points * right._2)
+}
 
 ~~~
 
 </div>
 </div>
 
-`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output Tuple. The order of indexes defines the order of fields in the output tuple.
-The join projection works also for non-Tuple DataSets. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output Tuple.
-
 #### Join with DataSet Size Hint
 
 In order to guide the optimizer to pick the right execution strategy, you can hint the size of a DataSet to join as shown here:
@@ -827,6 +856,14 @@ DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+val input1: DataSet[(Int, String)] = // [...]
+val input2: DataSet[(Int, String)] = // [...]
+
+// hint that the second DataSet is very small
+val result1 = input1.joinWithTiny(input2).where(0).equalTo(0)
+
+// hint that the second DataSet is very large
+val result1 = input1.joinWithHuge(input2).where(0).equalTo(0)
 
 ~~~
 
@@ -836,15 +873,15 @@ DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
 ### Cross
 
 The Cross transformation combines two DataSets into one DataSet. It builds all pairwise combinations of the elements of both input DataSets, i.e., it builds a Cartesian product.
-The Cross transformation either calls a user-defined `CrossFunction` on each pair of elements or applies a projection. Both modes are shown in the following.
+The Cross transformation either calls a user-defined cross function on each pair of elements or outputs a Tuple2. Both modes are shown in the following.
 
 **Note:** Cross is potentially a *very* compute-intensive operation which can challenge even large compute clusters!
 
 #### Cross with User-Defined Function
 
-A Cross transformation can call a user-defined `CrossFunction`. A `CrossFunction` receives one element of the first input and one element of the second input and returns exactly one result element.
+A Cross transformation can call a user-defined cross function. A cross function receives one element of the first input and one element of the second input and returns exactly one result element.
 
-The following code shows how to apply a Cross transformation on two DataSets using a `CrossFunction`:
+The following code shows how to apply a Cross transformation on two DataSets using a cross function:
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -877,23 +914,10 @@ DataSet<Tuple3<Integer, Integer, Double>>
                    .with(new EuclideanDistComputer());
 ~~~
 
-</div>
-<div data-lang="scala" markdown="1">
-
-~~~scala
-
-~~~
-
-</div>
-</div>
-
 #### Cross with Projection
 
 A Cross transformation can also construct result tuples using a projection as shown here:
 
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-
 ~~~java
 DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
 DataSet<Tuple2<Integer, Double>> input2 = // [...]
@@ -905,18 +929,28 @@ DataSet<Tuple4<Integer, Byte, Integer, Double>
                   .types(Integer.class, Byte.class, Integer.class, Double.class);
 ~~~
 
+The field selection in a Cross projection works the same way as in the projection of Join results.
+
 </div>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+case class Coord(id: Int, x: Int, y: Int)
+
+val coords1: DataSet[Coord] = // [...]
+val coords2: DataSet[Coord] = // [...]
 
+val distances = coords1.cross(coords2) {
+  (c1, c2) =>
+    val dist = sqrt(pow(c1.x - c2.x, 2) + pow(c1.y - c2.y, 2))
+    (c1.id, c2.id, dist)
+}
 ~~~
 
+
 </div>
 </div>
 
-The field selection in a Cross projection works the same way as in the projection of Join results.
-
 #### Cross with DataSet Size Hint
 
 In order to guide the optimizer to pick the right execution strategy, you can hint the size of a DataSet to cross as shown here:
@@ -947,6 +981,14 @@ DataSet<Tuple3<Integer, Integer, String>>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+val input1: DataSet[(Int, String)] = // [...]
+val input2: DataSet[(Int, String)] = // [...]
+
+// hint that the second DataSet is very small
+val result1 = input1.crossWithTiny(input2)
+
+// hint that the second DataSet is very large
+val result1 = input1.crossWithHuge(input2)
 
 ~~~
 
@@ -955,13 +997,14 @@ DataSet<Tuple3<Integer, Integer, String>>
 
 ### CoGroup
 
-The CoGroup transformation jointly processes groups of two DataSets. Both DataSets are grouped on a defined key and groups of both DataSets that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one DataSet has a group, the `CoGroupFunction` is called with this group and an empty group.
-A `CoGroupFunction` can separately iterate over the elements of both groups and return an arbitrary number of result elements.
+The CoGroup transformation jointly processes groups of two DataSets. Both DataSets are grouped on a defined key and groups of both DataSets that share the same key are handed together to a user-defined co-group function. If for a specific key only one DataSet has a group, the co-group function is called with this group and an empty group.
+A co-group function can separately iterate over the elements of both groups and return an arbitrary number of result elements.
 
 Similar to Reduce, GroupReduce, and Join, keys can be defined using
 
-- a `KeySelector` function or
-- one or more field position keys (Tuple DataSet only).
+- a key-selector function or
+- one or more field position keys (Tuple DataSet only) or
+- Case Class fields.
 
 #### CoGroup on DataSets Grouped by Field Position Keys (Tuple DataSets only)
 
@@ -1010,7 +1053,19 @@ DataSet<Double> output = iVals.coGroup(dVals)
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+val iVals: DataSet[(String, Int)] = // [...]
+val dVals: DataSet[(String, Double)] = // [...]
+
+val output = iVals.coGroup(dVals).where(0).equalTo(0) {
+  (iVals, dVals, out: Collector[Double]) =>
+    val ints = iVals map { _._2 } toSet
 
+    for (dVal <- dVals) {
+      for (i <- ints) {
+        out.collect(dVal._2 * i)
+      }
+    }
+}
 ~~~
 
 </div>
@@ -1031,15 +1086,18 @@ Produces the union of two DataSets, which have to be of the same type. A union o
 DataSet<Tuple2<String, Integer>> vals1 = // [...]
 DataSet<Tuple2<String, Integer>> vals2 = // [...]
 DataSet<Tuple2<String, Integer>> vals3 = // [...]
-DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2)
-                    .union(vals3);
+DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2).union(vals3);
 ~~~
 
 </div>
 <div data-lang="scala" markdown="1">
 
 ~~~scala
+val vals1: DataSet[(String, Int)] = // [...]
+val vals2: DataSet[(String, Int)] = // [...]
+val vals3: DataSet[(String, Int)] = // [...]
 
+val unioned = vals1.union(vals2).union(vals3)
 ~~~
 
 </div>


[06/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
new file mode 100644
index 0000000..05f9917
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
@@ -0,0 +1,230 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala
+
+import org.apache.commons.lang3.Validate
+import org.apache.flink.api.common.InvalidProgramException
+import org.apache.flink.api.common.functions.{RichCoGroupFunction, CoGroupFunction}
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.Collector
+
+import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
+
+
+/**
+ * A specific [[DataSet]] that results from a `coGroup` operation. The result of a default coGroup is
+ * a tuple containing two arrays of values from the two sides of the coGroup. The result of the
+ * coGroup can be changed by specifying a custom coGroup function using the `apply` method or by
+ * providing a [[RichCoGroupFunction]].
+ *
+ * Example:
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val coGroupResult = left.coGroup(right).where(0, 2).isEqualTo(0, 1) {
+ *     (left, right) => new MyCoGroupResult(left.min, right.max)
+ *   }
+ * }}}
+ *
+ * Or, using key selector functions with tuple data types:
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val coGroupResult = left.coGroup(right).where({_._1}).isEqualTo({_._1) {
+ *     (left, right) => new MyCoGroupResult(left.max, right.min)
+ *   }
+ * }}}
+ *
+ * @tparam T Type of the left input of the coGroup.
+ * @tparam O Type of the right input of the coGroup.
+ */
+trait CoGroupDataSet[T, O] extends DataSet[(Array[T], Array[O])] {
+
+  /**
+   * Creates a new [[DataSet]] where the result for each pair of co-grouped element lists is the
+   * result of the given function. You can either return an element or choose to return [[None]],
+   * which allows implementing a filter directly in the coGroup function.
+   */
+  def apply[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], TraversableOnce[O]) => Option[R]): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] where the result for each pair of co-grouped element lists is the
+   * result of the given function. The function can output zero or more elements using the
+   * [[Collector]] which will form the result.
+   */
+  def apply[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], TraversableOnce[O], Collector[R]) => Unit): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing each pair of co-grouped element lists to the given
+   * function. The function can output zero or more elements using the [[Collector]] which will form
+   * the result.
+   *
+   * A [[RichCoGroupFunction]] can be used to access the
+   * broadcast variables and the [[org.apache.flink.api.common.functions.RuntimeContext]].
+   */
+  def apply[R: TypeInformation: ClassTag](joiner: CoGroupFunction[T, O, R]): DataSet[R]
+}
+
+/**
+ * Private implementation for [[CoGroupDataSet]] to keep the implementation details, i.e. the
+ * parameters of the constructor, hidden.
+ */
+private[flink] class CoGroupDataSetImpl[T, O](
+    coGroupOperator: CoGroupOperator[T, O, (Array[T], Array[O])],
+    thisSet: JavaDataSet[T],
+    otherSet: JavaDataSet[O],
+    thisKeys: Keys[T],
+    otherKeys: Keys[O]) extends DataSet(coGroupOperator) with CoGroupDataSet[T, O] {
+
+  def apply[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], TraversableOnce[O]) => Option[R]): DataSet[R] = {
+    Validate.notNull(fun, "CoGroup function must not be null.")
+    val coGrouper = new CoGroupFunction[T, O, R] {
+      def coGroup(left: java.lang.Iterable[T], right: java.lang.Iterable[O], out: Collector[R]) = {
+        fun(left.iterator.asScala, right.iterator.asScala) map { out.collect(_) }
+      }
+    }
+    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, coGrouper, implicitly[TypeInformation[R]])
+    wrap(coGroupOperator)
+  }
+
+  def apply[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], TraversableOnce[O], Collector[R]) => Unit): DataSet[R] = {
+    Validate.notNull(fun, "CoGroup function must not be null.")
+    val coGrouper = new CoGroupFunction[T, O, R] {
+      def coGroup(left: java.lang.Iterable[T], right: java.lang.Iterable[O], out: Collector[R]) = {
+        fun(left.iterator.asScala, right.iterator.asScala, out)
+      }
+    }
+    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, coGrouper, implicitly[TypeInformation[R]])
+    wrap(coGroupOperator)
+  }
+
+  def apply[R: TypeInformation: ClassTag](joiner: CoGroupFunction[T, O, R]): DataSet[R] = {
+    Validate.notNull(joiner, "CoGroup function must not be null.")
+    val coGroupOperator = new CoGroupOperator[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, joiner, implicitly[TypeInformation[R]])
+    wrap(coGroupOperator)
+  }
+}
+
+/**
+ * An unfinished coGroup operation that results from [[DataSet.coGroup()]] The keys for the left and
+ * right side must be specified using first `where` and then `isEqualTo`. For example:
+ *
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val joinResult = left.coGroup(right).where(...).isEqualTo(...)
+ * }}}
+ * @tparam T The type of the left input of the coGroup.
+ * @tparam O The type of the right input of the coGroup.
+ */
+trait UnfinishedCoGroupOperation[T, O]
+  extends UnfinishedKeyPairOperation[T, O, CoGroupDataSet[T, O]]
+
+/**
+ * Private implementation for [[UnfinishedCoGroupOperation]] to keep the implementation details,
+ * i.e. the parameters of the constructor, hidden.
+ */
+private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
+    leftSet: JavaDataSet[T],
+    rightSet: JavaDataSet[O])
+  extends UnfinishedKeyPairOperation[T, O, CoGroupDataSet[T, O]](leftSet, rightSet)
+  with UnfinishedCoGroupOperation[T, O] {
+
+  private[flink] def finish(leftKey: Keys[T], rightKey: Keys[O]) = {
+    val coGrouper = new CoGroupFunction[T, O, (Array[T], Array[O])] {
+      def coGroup(
+                   left: java.lang.Iterable[T],
+                   right: java.lang.Iterable[O],
+                   out: Collector[(Array[T], Array[O])]) = {
+        val leftResult = Array[Any](left.asScala.toSeq: _*).asInstanceOf[Array[T]]
+        val rightResult = Array[Any](right.asScala.toSeq: _*).asInstanceOf[Array[O]]
+
+        out.collect((leftResult, rightResult))
+      }
+    }
+
+    // We have to use this hack, for some reason classOf[Array[T]] does not work.
+    // Maybe because ObjectArrayTypeInfo does not accept the Scala Array as an array class.
+    val leftArrayType = ObjectArrayTypeInfo.getInfoFor(new Array[T](0).getClass, leftSet.getType)
+    val rightArrayType = ObjectArrayTypeInfo.getInfoFor(new Array[O](0).getClass, rightSet.getType)
+
+    val returnType = new ScalaTupleTypeInfo[(Array[T], Array[O])](
+      classOf[(Array[T], Array[O])], Seq(leftArrayType, rightArrayType)) {
+
+      override def createSerializer: TypeSerializer[(Array[T], Array[O])] = {
+        val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
+        for (i <- 0 until getArity()) {
+          fieldSerializers(i) = types(i).createSerializer
+        }
+
+        new ScalaTupleSerializer[(Array[T], Array[O])](
+          classOf[(Array[T], Array[O])],
+          fieldSerializers) {
+          override def createInstance(fields: Array[AnyRef]) = {
+            (fields(0).asInstanceOf[Array[T]], fields(1).asInstanceOf[Array[O]])
+          }
+        }
+      }
+    }
+    val coGroupOperator = new CoGroupOperator[T, O, (Array[T], Array[O])](
+      leftSet, rightSet, leftKey, rightKey, coGrouper, returnType)
+
+    // sanity check solution set key mismatches
+    leftSet match {
+      case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
+        leftKey match {
+          case keyFields: Keys.FieldPositionKeys[_] =>
+            val positions: Array[Int] = keyFields.computeLogicalKeyPositions
+            solutionSet.checkJoinKeyFields(positions)
+          case _ =>
+            throw new InvalidProgramException("Currently, the solution set may only be joined " +
+              "with " +
+              "using tuple field positions.")
+        }
+      case _ =>
+    }
+    rightSet match {
+      case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
+        rightKey match {
+          case keyFields: Keys.FieldPositionKeys[_] =>
+            val positions: Array[Int] = keyFields.computeLogicalKeyPositions
+            solutionSet.checkJoinKeyFields(positions)
+          case _ =>
+            throw new InvalidProgramException("Currently, the solution set may only be joined " +
+              "with " +
+              "using tuple field positions.")
+        }
+      case _ =>
+    }
+
+    new CoGroupDataSetImpl(coGroupOperator, leftSet, rightSet, leftKey, rightKey)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
index 5a53a85..d538188 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
@@ -19,7 +19,7 @@
 
 package org.apache.flink.api.scala.codegen
 
-class Counter {
+private[flink] class Counter {
   private var value: Int = 0
 
   def next: Int = {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/DeserializeMethodGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/DeserializeMethodGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/DeserializeMethodGen.scala
deleted file mode 100644
index 1362d3f..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/DeserializeMethodGen.scala
+++ /dev/null
@@ -1,261 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.reflect.macros.Context
-
-trait DeserializeMethodGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with TreeGen[C] with SerializerGen[C] with Loggers[C] =>
-  import c.universe._
-
-  protected def mkDeserialize(desc: UDTDescriptor, listImpls: Map[Int, Type]): List[Tree] = {
-
-//    val rootRecyclingOn = mkMethod("deserializeRecyclingOn", Flag.OVERRIDE | Flag.FINAL, List(("record", typeOf[org.apache.flink.pact.common.`type`.Record])), desc.tpe, {
-    val rootRecyclingOn = mkMethod("deserializeRecyclingOn", Flag.FINAL, List(("record", typeOf[org.apache.flink.types.Record])), desc.tpe, {
-      val env = GenEnvironment(listImpls, "flat" + desc.id, false, true, true, true)
-      mkSingle(genDeserialize(desc, Ident("record"), env, Map()))
-    })
-
-//    val rootRecyclingOff = mkMethod("deserializeRecyclingOff", Flag.OVERRIDE | Flag.FINAL, List(("record", typeOf[org.apache.flink.pact.common.`type`.Record])), desc.tpe, {
-    val rootRecyclingOff = mkMethod("deserializeRecyclingOff", Flag.FINAL, List(("record", typeOf[org.apache.flink.types.Record])), desc.tpe, {
-      val env = GenEnvironment(listImpls, "flat" + desc.id, false, false, true, true)
-      mkSingle(genDeserialize(desc, Ident("record"), env, Map()))
-    })
-
-    val aux = desc.getRecursiveRefs map { desc =>
-      mkMethod("deserialize" + desc.id, Flag.PRIVATE | Flag.FINAL, List(("record", typeOf[org.apache.flink.types.Record])), desc.tpe, {
-        val env = GenEnvironment(listImpls, "boxed" + desc.id, true, false, false, true)
-        mkSingle(genDeserialize(desc, Ident("record"), env, Map()))
-      })
-    }
-
-    rootRecyclingOn +: rootRecyclingOff +: aux.toList
-  }
-
-  private def genDeserialize(desc: UDTDescriptor, source: Tree, env: GenEnvironment, scope: Map[Int, (String, Type)]): Seq[Tree] = desc match {
-
-    case PactValueDescriptor(id, tpe) => {
-      val chk = env.mkChkIdx(id)
-      val get = env.mkGetField(id, source, tpe)
-
-      Seq(mkIf(chk, get, mkNull))
-    }
-    
-    case PrimitiveDescriptor(id, _, default, _) => {
-      val chk = env.mkChkIdx(id)
-      val des = env.mkGetFieldInto(id, source)
-      val get = env.mkGetValue(id)
-
-      Seq(mkIf(chk, Block(List(des), get), default))
-    }
-
-    case BoxedPrimitiveDescriptor(id, tpe, _, _, box, _) => {
-      val des = env.mkGetFieldInto(id, source)
-      val chk = mkAnd(env.mkChkIdx(id), des)
-      val get = box(env.mkGetValue(id))
-
-      Seq(mkIf(chk, get, mkNull))
-    }
-
-    case list @ ListDescriptor(id, tpe, _, elem) => {
-      val chk = mkAnd(env.mkChkIdx(id), env.mkNotIsNull(id, source))
-
-      val (init, pactList) = env.reentrant match {
-
-        // This is a bit conservative, but avoids runtime checks
-        // and/or even more specialized deserialize() methods to
-        // track whether it's safe to reuse the list variable.
-        case true => {
-          val listTpe = env.listImpls(id)
-          val list = mkVal("list" + id, NoFlags, false, listTpe, New(TypeTree(listTpe), List(List())))
-          (list, Ident("list" + id: TermName))
-        }
-
-        case false => {
-          val clear = Apply(Select(env.mkSelectWrapper(id), "clear"), List())
-          (clear, env.mkSelectWrapper(id))
-        }
-      }
-
-      //        val buildTpe = appliedType(builderClass.tpe, List(elem.tpe, tpe))
-      //        val build = mkVal(env.methodSym, "b" + id, 0, false, buildTpe) { _ => Apply(Select(cbf(), "apply"), List()) }
-//      val userList = mkVal("b" + id, NoFlags, false, tpe, New(TypeTree(tpe), List(List())))
-      val buildTpe = mkBuilderOf(elem.tpe, tpe)
-      val cbf = c.inferImplicitValue(mkCanBuildFromOf(tpe, elem.tpe, tpe))
-      val build = mkVal("b" + id, NoFlags, false, buildTpe, Apply(Select(cbf, "apply": TermName), List()))
-      val des = env.mkGetFieldInto(id, source, pactList)
-      val body = genDeserializeList(elem, pactList, Ident("b" + id: TermName), env.copy(allowRecycling = false, chkNull = true), scope)
-      val stats = init +: des +: build +: body
-
-      Seq(mkIf(chk, Block(stats.init.toList, stats.last), mkNull))
-    }
-
-    // we have a mutable UDT and the context allows recycling
-    case CaseClassDescriptor(_, tpe, true, _, getters) if env.allowRecycling => {
-
-      val fields = getters filterNot { _.isBaseField } map {
-        case FieldAccessor(_, _, _, _, desc) => (desc.id, mkVal("v" + desc.id, NoFlags, false, desc.tpe, {
-          mkSingle(genDeserialize(desc, source, env, scope))
-        }), desc.tpe, "v" + desc.id)
-      }
-
-      val newScope = scope ++ (fields map { case (id, tree, tpe, name) => id -> (name, tpe) })
-
-      val stats = fields map { _._2 }
-
-      val setterStats = getters map {
-        case FieldAccessor(_, setter, fTpe, _, fDesc) => {
-          val (name, tpe) = newScope(fDesc.id)
-          val castVal = maybeMkAsInstanceOf(Ident(name: TermName))(c.WeakTypeTag(tpe), c.WeakTypeTag(fTpe))
-          env.mkCallSetMutableField(desc.id, setter, castVal)
-        }
-      }
-
-      val ret = env.mkSelectMutableUdtInst(desc.id)
-
-      (stats ++ setterStats) :+ ret
-    }
-
-    case CaseClassDescriptor(_, tpe, _, _, getters) => {
-
-      val fields = getters filterNot { _.isBaseField } map {
-        case FieldAccessor(_, _, _, _, desc) => (desc.id, mkVal("v" + desc.id, NoFlags, false, desc.tpe, {
-          mkSingle(genDeserialize(desc, source, env, scope))
-        }), desc.tpe, "v" + desc.id)
-      }
-
-      val newScope = scope ++ (fields map { case (id, tree, tpe, name) => id -> (name, tpe) })
-
-      val stats = fields map { _._2 }
-
-      val args = getters map {
-        case FieldAccessor(_, _, fTpe, _, fDesc) => {
-          val (name, tpe) = newScope(fDesc.id)
-          maybeMkAsInstanceOf(Ident(name: TermName))(c.WeakTypeTag(tpe), c.WeakTypeTag(fTpe))
-        }
-      }
-
-      val ret = New(TypeTree(tpe), List(args.toList))
-
-      stats :+ ret
-    }
-
-    case BaseClassDescriptor(_, tpe, Seq(tagField, baseFields @ _*), subTypes) => {
-
-      val fields = baseFields map {
-        case FieldAccessor(_, _, _, _, desc) => (desc.id, mkVal("v" + desc.id, NoFlags, false, desc.tpe, {
-          val special = desc match {
-            case d @ PrimitiveDescriptor(id, _, _, _) if id == tagField.desc.id => d.copy(default = Literal(Constant(-1)))
-            case _ => desc
-          }
-          mkSingle(genDeserialize(desc, source, env, scope))
-        }), desc.tpe, "v" + desc.id)
-      }
-
-      val newScope = scope ++ (fields map { case (id, tree, tpe, name) => id -> (name, tpe) })
-
-      val stats = fields map { _._2 }
-
-      val cases = subTypes.zipWithIndex.toList map {
-        case (dSubType, i) => {
-          val code = mkSingle(genDeserialize(dSubType, source, env, newScope))
-          val pat = Bind("tag": TermName, Literal(Constant(i)))
-          CaseDef(pat, EmptyTree, code)
-        }
-      }
-
-      val chk = env.mkChkIdx(tagField.desc.id)
-      val des = env.mkGetFieldInto(tagField.desc.id, source)
-      val get = env.mkGetValue(tagField.desc.id)
-      Seq(mkIf(chk, Block(stats.toList :+ des, Match(get, cases)), mkNull))
-    }
-
-    case RecursiveDescriptor(id, tpe, refId) => {
-      val chk = mkAnd(env.mkChkIdx(id), env.mkNotIsNull(id, source))
-      val rec = mkVal("record" + id, NoFlags, false, typeOf[org.apache.flink.types.Record], New(TypeTree(typeOf[org.apache.flink.types.Record]), List(List())))
-      val get = env.mkGetFieldInto(id, source, Ident("record" + id: TermName))
-      val des = env.mkCallDeserialize(refId, Ident("record" + id: TermName))
-
-      Seq(mkIf(chk, Block(List(rec, get), des), mkNull))
-    }
-
-    case _ => Seq(mkNull)
-  }
-
-  private def genDeserializeList(elem: UDTDescriptor, source: Tree, target: Tree, env: GenEnvironment, scope: Map[Int, (String, Type)]): Seq[Tree] = {
-
-    val size = mkVal("size", NoFlags, false, definitions.IntTpe, Apply(Select(source, "size"), List()))
-    val sizeHint = Apply(Select(target, "sizeHint"), List(Ident("size": TermName)))
-    val i = mkVar("i", NoFlags, false, definitions.IntTpe, mkZero)
-
-    val loop = mkWhile(Apply(Select(Ident("i": TermName), "$less"), List(Ident("size": TermName)))) {
-
-      val item = mkVal("item", NoFlags, false, getListElemWrapperType(elem, env), Apply(Select(source, "get"), List(Ident("i": TermName))))
-
-      val (stats, value) = elem match {
-
-        case PrimitiveDescriptor(_, _, _, wrapper) => (Seq(), env.mkGetValue(Ident("item": TermName)))
-
-        case BoxedPrimitiveDescriptor(_, _, _, wrapper, box, _) => (Seq(), box(env.mkGetValue(Ident("item": TermName))))
-        
-        case PactValueDescriptor(_, tpe) => (Seq(), Ident("item": TermName))
-
-        case ListDescriptor(id, tpe, _, innerElem) => {
-
-          //            val buildTpe = appliedType(builderClass.tpe, List(innerElem.tpe, tpe))
-          //            val build = mkVal(env.methodSym, "b" + id, 0, false, buildTpe) { _ => Apply(Select(cbf(), "apply"), List()) }
-          val buildTpe = mkBuilderOf(innerElem.tpe, tpe)
-          val cbf = c.inferImplicitValue(mkCanBuildFromOf(tpe, innerElem.tpe, tpe))
-          val build = mkVal("b" + id, NoFlags, false, buildTpe, Apply(Select(cbf, "apply": TermName), List()))
-          val body = mkVal("v" + id, NoFlags, false, elem.tpe,
-            mkSingle(genDeserializeList(innerElem, Ident("item": TermName), Ident("b" + id: TermName), env, scope)))
-          (Seq(build, body), Ident("v" + id: TermName))
-        }
-
-        case RecursiveDescriptor(id, tpe, refId) => (Seq(), env.mkCallDeserialize(refId, Ident("item": TermName)))
-
-        case _ => {
-          val body = genDeserialize(elem, Ident("item": TermName), env.copy(idxPrefix = "boxed" + elem.id, chkIndex = false, chkNull = false), scope)
-          val v = mkVal("v" + elem.id, NoFlags, false, elem.tpe, mkSingle(body))
-          (Seq(v), Ident("v" + elem.id: TermName))
-        }
-      }
-
-      val chk = env.mkChkNotNull(Ident("item": TermName), elem.tpe)
-      val add = Apply(Select(target, "$plus$eq"), List(value))
-      val addNull = Apply(Select(target, "$plus$eq"), List(mkNull))
-      val inc = Assign(Ident("i": TermName), Apply(Select(Ident("i": TermName), "$plus"), List(mkOne)))
-
-      Block(List(item, mkIf(chk, mkSingle(stats :+ add), addNull)), inc)
-    }
-    
-    val get = Apply(Select(target, "result"), List())
-
-    Seq(size, sizeHint, i, loop, get)
-  }
-  
-
-  private def getListElemWrapperType(desc: UDTDescriptor, env: GenEnvironment): Type = desc match {
-    case PrimitiveDescriptor(_, _, _, wrapper) => wrapper
-    case BoxedPrimitiveDescriptor(_, _, _, wrapper, _, _) => wrapper
-    case PactValueDescriptor(_, tpe) => tpe
-    case ListDescriptor(id, _, _, _) => env.listImpls(id)
-    case _ => typeOf[org.apache.flink.types.Record]
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Logger.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Logger.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Logger.scala
deleted file mode 100644
index c1f98ae..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Logger.scala
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.reflect.macros.Context
-
-trait Loggers[C <: Context] { this: MacroContextHolder[C] =>
-  import c.universe._
-
-  abstract sealed class LogLevel extends Ordered[LogLevel] {
-    protected[Loggers] val toInt: Int
-    override def compare(that: LogLevel) = this.toInt.compare(that.toInt)
-  }
-
-  object LogLevel {
-    def unapply(name: String): Option[LogLevel] = name match {
-      case "error" | "Error"     => Some(Error)
-      case "warn" | "Warn"       => Some(Warn)
-      case "debug" | "Debug"     => Some(Debug)
-      case "inspect" | "Inspect" => Some(Inspect)
-      case _                     => None
-    }
-    case object Error extends LogLevel { override val toInt = 1 }
-    case object Warn extends LogLevel { override val toInt = 2 }
-    case object Debug extends LogLevel { override val toInt = 3 }
-    case object Inspect extends LogLevel { override val toInt = 4 }
-  }
-
-  object logger { var level: LogLevel = LogLevel.Warn }
-  private val counter = new Counter
-
-  trait Logger {
-
-    abstract sealed class Severity {
-      protected val toInt: Int
-      protected def reportInner(msg: String, pos: Position)
-
-      protected def formatMsg(msg: String) = msg
-
-      def isEnabled = this.toInt <= logger.level.toInt
-
-      def report(msg: String) = {
-        if (isEnabled) {
-          reportInner(formatMsg(msg), c.enclosingPosition)
-        }
-      }
-    }
-
-    case object Error extends Severity {
-      override val toInt = LogLevel.Error.toInt
-      override def reportInner(msg: String, pos: Position) = c.error(pos, msg)
-    }
-
-    case object Warn extends Severity {
-      override val toInt = LogLevel.Warn.toInt
-      override def reportInner(msg: String, pos: Position) = c.warning(pos, msg)
-    }
-
-    case object Debug extends Severity {
-      override val toInt = LogLevel.Debug.toInt
-      override def reportInner(msg: String, pos: Position) = c.info(pos, msg, true)
-    }
-
-    def getMsgAndStackLine(e: Throwable) = {
-      val lines = e.getStackTrace.map(_.toString)
-      val relevant = lines filter { _.contains("org.apache.flink") }
-      val stackLine = relevant.headOption getOrElse e.getStackTrace.toString
-      e.getMessage() + " @ " + stackLine
-    }
-
-    def posString(pos: Position): String = pos match {
-      case NoPosition => "?:?"
-      case _          => pos.line + ":" + pos.column
-    }
-
-    def safely(default: => Tree, inspect: Boolean)(onError: Throwable => String)(block: => Tree): Tree = {
-      try {
-        block
-      } catch {
-        case e:Throwable => {
-          Error.report(onError(e));
-          val ret = default
-          ret
-        }
-      }
-    }
-
-    def verbosely[T](obs: T => String)(block: => T): T = {
-      val ret = block
-      Debug.report(obs(ret))
-      ret
-    }
-
-    def maybeVerbosely[T](guard: T => Boolean)(obs: T => String)(block: => T): T = {
-      val ret = block
-      if (guard(ret)) Debug.report(obs(ret))
-      ret
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
index effc27b..4ce4922 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
@@ -16,22 +16,16 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.scala.codegen
 
 import scala.reflect.macros.Context
 
-class MacroContextHolder[C <: Context](val c: C)
+private[flink] class MacroContextHolder[C <: Context](val c: C)
 
-object MacroContextHolder {
+private[flink] object MacroContextHolder {
   def newMacroHelper[C <: Context](c: C) = new MacroContextHolder[c.type](c)
-    	with Loggers[c.type]
-    	with UDTDescriptors[c.type]
-    	with UDTAnalyzer[c.type]
+    	with TypeDescriptors[c.type]
+    	with TypeAnalyzer[c.type]
     	with TreeGen[c.type]
-    	with SerializerGen[c.type]
-    	with SerializeMethodGen[c.type]
-    	with DeserializeMethodGen[c.type]
-    	with UDTGen[c.type]
-    	with SelectionExtractor[c.type]
+      with TypeInformationGen[c.type]
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SelectionExtractor.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SelectionExtractor.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SelectionExtractor.scala
deleted file mode 100644
index 37abd27..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SelectionExtractor.scala
+++ /dev/null
@@ -1,184 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.reflect.macros.Context
-import scala.Option.option2Iterable
-
-import org.apache.flink.api.scala.analysis.FieldSelector
-
-trait SelectionExtractor[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with UDTAnalyzer[C] with Loggers[C] with TreeGen[C] =>
-  import c.universe._
-
-  def getSelector[T: c.WeakTypeTag, R: c.WeakTypeTag](fun: c.Expr[T => R]): Expr[List[Int]] =
-    (new SelectionExtractorInstance with Logger).extract(fun)
-
-  class SelectionExtractorInstance { this: Logger =>
-    def extract[T: c.WeakTypeTag, R: c.WeakTypeTag](fun: c.Expr[T => R]): Expr[List[Int]] = {
-      val result = getSelector(fun.tree) match {
-        case Left(errs) => Left(errs.toList)
-        case Right(sels) => getUDTDescriptor(weakTypeOf[T]) match {
-          case UnsupportedDescriptor(id, tpe, errs) => Left(errs.toList)
-          case desc: UDTDescriptor => chkSelectors(desc, sels) match {
-            case Nil => Right(desc, sels map { _.tail })
-            case errs => Left(errs)
-          }
-        }
-      }
-
-      result match {
-        case Left(errs) => {
-          errs foreach { err => c.error(c.enclosingPosition, s"Error analyzing FieldSelector ${show(fun.tree)}: " + err) }
-          reify { throw new RuntimeException("Invalid key selector."); }
-        }
-        case Right((udtDesc, sels)) => {
-          val descs: List[Option[UDTDescriptor]] = sels flatMap { sel: List[String] => udtDesc.select(sel) }
-          descs foreach { desc => desc map { desc => if (!desc.canBeKey) c.error(c.enclosingPosition, "Type " + desc.tpe + " cannot be key.") } }
-          val ids = descs map { _ map { _.id } }
-          ids forall { _.isDefined } match {
-            case false => {
-              c.error(c.enclosingPosition, s"Could not determine ids of key fields: ${ids}")
-              reify { throw new RuntimeException("Invalid key selector."); }
-            }
-            case true => {
-              val generatedIds = ids map { _.get } map { id => Literal(Constant(id: Int)) }
-              val generatedList = mkList(generatedIds)
-              reify {
-                val list = c.Expr[List[Int]](generatedList).splice
-                list
-              }
-            }
-          }
-        }
-      }
-      
-    }
-
-    private def getSelector(tree: Tree): Either[List[String], List[List[String]]] = tree match {
-
-      case Function(List(p), body) => getSelector(body, Map(p.symbol -> Nil)) match {
-        case err @ Left(_) => err
-        case Right(sels) => Right(sels map { sel => p.name.toString +: sel })
-      }
-
-      case _ => Left(List("expected lambda expression literal but found " + show(tree)))
-    }
-
-    private def getSelector(tree: Tree, roots: Map[Symbol, List[String]]): Either[List[String], List[List[String]]] = tree match {
-
-      case SimpleMatch(body, bindings) => getSelector(body, roots ++ bindings)
-
-      case Match(_, List(CaseDef(pat, EmptyTree, _))) => Left(List("case pattern is too complex"))
-      case Match(_, List(CaseDef(_, guard, _))) => Left(List("case pattern is guarded"))
-      case Match(_, _ :: _ :: _) => Left(List("match contains more than one case"))
-
-      case TupleCtor(args) => {
-
-        val (errs, sels) = args.map(arg => getSelector(arg, roots)).partition(_.isLeft)
-
-        errs match {
-          case Nil => Right(sels.map(_.right.get).flatten)
-          case _ => Left(errs.map(_.left.get).flatten)
-        }
-      }
-
-      case Apply(tpt@TypeApply(_, _), _) => Left(List("constructor call on non-tuple type " + tpt.tpe))
-
-      case Ident(name) => roots.get(tree.symbol) match {
-        case Some(sel) => Right(List(sel))
-        case None => Left(List("unexpected identifier " + name))
-      }
-
-      case Select(src, member) => getSelector(src, roots) match {
-        case err @ Left(_) => err
-        case Right(List(sel)) => Right(List(sel :+ member.toString))
-        case _ => Left(List("unsupported selection"))
-      }
-
-      case _ => Left(List("unsupported construct of kind " + showRaw(tree)))
-
-    }
-
-    private object SimpleMatch {
-
-      def unapply(tree: Tree): Option[(Tree, Map[Symbol, List[String]])] = tree match {
-
-        case Match(arg, List(cd @ CaseDef(CasePattern(bindings), EmptyTree, body))) => Some((body, bindings))
-        case _ => None
-      }
-
-      private object CasePattern {
-
-        def unapply(tree: Tree): Option[Map[Symbol, List[String]]] = tree match {
-
-          case Apply(MethodTypeTree(params), binds) => {
-            val exprs = params.zip(binds) map {
-              case (p, CasePattern(inners)) => Some(inners map { case (sym, path) => (sym, p.name.toString +: path) })
-              case _ => None
-            }
-            if (exprs.forall(_.isDefined)) {
-              Some(exprs.flatten.flatten.toMap)
-            }
-            else
-              None
-          }
-
-          case Ident(_) | Bind(_, Ident(_)) => Some(Map(tree.symbol -> Nil))
-          case Bind(_, CasePattern(inners)) => Some(inners + (tree.symbol -> Nil))
-          case _ => None
-        }
-      }
-
-      private object MethodTypeTree {
-        def unapply(tree: Tree): Option[List[Symbol]] = tree match {
-          case _: TypeTree => tree.tpe match {
-            case MethodType(params, _) => Some(params)
-            case _ => None
-          }
-          case _ => None
-        }
-      }
-    }
-
-    private object TupleCtor {
-
-      def unapply(tree: Tree): Option[List[Tree]] = tree match {
-        case Apply(tpt@TypeApply(_, _), args) if isTupleTpe(tpt.tpe) => Some(args)
-        case _ => None
-      }
-
-      private def isTupleTpe(tpe: Type): Boolean = definitions.TupleClass.contains(tpe.typeSymbol)
-    }
-  }
-
-  protected def chkSelectors(udt: UDTDescriptor, sels: List[List[String]]): List[String] = {
-    sels flatMap { sel => chkSelector(udt, sel.head, sel.tail) }
-  }
-
-  protected def chkSelector(udt: UDTDescriptor, path: String, sel: List[String]): Option[String] = (udt, sel) match {
-    case (_, Nil) if udt.isPrimitiveProduct => None
-    case (_, Nil) => Some(path + ": " + udt.tpe + " is not a primitive or product of primitives")
-    case (_, field :: rest) => udt.select(field) match {
-      case None => Some("member " + field + " is not a case accessor of " + path + ": " + udt.tpe)
-      case Some(udt) => chkSelector(udt, path + "." + field, rest)
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializeMethodGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializeMethodGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializeMethodGen.scala
deleted file mode 100644
index 6424ea1..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializeMethodGen.scala
+++ /dev/null
@@ -1,226 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.reflect.macros.Context
-
-trait SerializeMethodGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with TreeGen[C] with SerializerGen[C] with Loggers[C] =>
-  import c.universe._
-
-  protected def mkSerialize(desc: UDTDescriptor, listImpls: Map[Int, Type]): List[Tree] = {
-
-//    val root = mkMethod("serialize", Flag.OVERRIDE | Flag.FINAL, List(("item", desc.tpe), ("record", typeOf[org.apache.flink.pact.common.`type`.Record])), definitions.UnitTpe, {
-    val root = mkMethod("serialize", Flag.FINAL, List(("item", desc.tpe), ("record", typeOf[org.apache.flink.types.Record])), definitions.UnitTpe, {
-      val env = GenEnvironment(listImpls, "flat" + desc.id, false, true, true, true)
-      val stats = genSerialize(desc, Ident("item": TermName), Ident("record": TermName), env)
-      Block(stats.toList, mkUnit)
-    })
-
-    val aux = desc.getRecursiveRefs map { desc =>
-      mkMethod("serialize" + desc.id, Flag.PRIVATE | Flag.FINAL, List(("item", desc.tpe), ("record", typeOf[org.apache.flink.types.Record])), definitions.UnitTpe, {
-        val env = GenEnvironment(listImpls, "boxed" + desc.id, true, false, false, true)
-        val stats = genSerialize(desc, Ident("item": TermName), Ident("record": TermName), env)
-        Block(stats.toList, mkUnit)
-      })
-    }
-
-    root +: aux.toList
-  }
-
-  private def genSerialize(desc: UDTDescriptor, source: Tree, target: Tree, env: GenEnvironment): Seq[Tree] = desc match {
-
-    case PactValueDescriptor(id, _) => {
-      val chk = env.mkChkIdx(id)
-      val set = env.mkSetField(id, target, source)
-
-      Seq(mkIf(chk, set))
-    }
-    
-    case PrimitiveDescriptor(id, _, _, _) => {
-      val chk = env.mkChkIdx(id)
-      val ser = env.mkSetValue(id, source)
-      val set = env.mkSetField(id, target)
-
-      Seq(mkIf(chk, Block(List(ser), set)))
-    }
-
-    case BoxedPrimitiveDescriptor(id, tpe, _, _, _, unbox) => {
-      val chk = mkAnd(env.mkChkIdx(id), env.mkChkNotNull(source, tpe))
-      val ser = env.mkSetValue(id, unbox(source))
-      val set = env.mkSetField(id, target)
-
-      Seq(mkIf(chk, Block(List(ser), set)))
-    }
-
-    case desc @ ListDescriptor(id, tpe, iter, elem) => {
-      val chk = mkAnd(env.mkChkIdx(id), env.mkChkNotNull(source, tpe))
-
-      val upd = desc.getInnermostElem match {
-        case _: RecursiveDescriptor => Some(Apply(Select(target, "updateBinaryRepresenation"), List()))
-        case _ => None
-      }
-
-      val (init, list) = env.reentrant match {
-
-        // This is a bit conservative, but avoids runtime checks
-        // and/or even more specialized serialize() methods to
-        // track whether it's safe to reuse the list variable.
-        case true => {
-          val listTpe = env.listImpls(id)
-          val list = mkVal("list" + id, NoFlags, false, listTpe, New(TypeTree(listTpe), List(List())))
-          (list, Ident("list" + id: TermName))
-        }
-
-        case false => {
-          val clear = Apply(Select(env.mkSelectWrapper(id), "clear"), List())
-          (clear, env.mkSelectWrapper(id))
-        }
-      }
-
-      val body = genSerializeList(elem, iter(source), list, env.copy(chkNull = true))
-      val set = env.mkSetField(id, target, list)
-      val stats = (init +: body) :+ set
-
-      val updStats = upd ++ stats
-      Seq(mkIf(chk, Block(updStats.init.toList, updStats.last)))
-    }
-
-    case CaseClassDescriptor(_, tpe, _, _, getters) => {
-      val chk = env.mkChkNotNull(source, tpe)
-      val stats = getters filterNot { _.isBaseField } flatMap { case FieldAccessor(sym, _, _, _, desc) => genSerialize(desc, Select(source, sym), target, env.copy(chkNull = true)) }
-
-      stats match {
-        case Nil => Seq()
-        case _ => Seq(mkIf(chk, mkSingle(stats)))
-      }
-    }
-
-    case BaseClassDescriptor(id, tpe, Seq(tagField, baseFields @ _*), subTypes) => {
-      val chk = env.mkChkNotNull(source, tpe)
-      val fields = baseFields flatMap { (f => genSerialize(f.desc, Select(source, f.getter), target, env.copy(chkNull = true))) }
-      val cases = subTypes.zipWithIndex.toList map {
-        case (dSubType, i) => {
-
-          val pat = Bind("inst": TermName, Typed(Ident("_"), TypeTree(dSubType.tpe)))
-          val cast = None
-          val inst = Ident("inst": TermName)
-          //            val (pat, cast, inst) = {
-          //              val erasedTpe = mkErasedType(env.methodSym, dSubType.tpe)
-          //
-          //              if (erasedTpe =:= dSubType.tpe) {
-          //
-          //                val pat = Bind(newTermName("inst"), Typed(Ident("_"), TypeTree(dSubType.tpe)))
-          //                (pat, None, Ident(newTermName("inst")))
-          //
-          //              } else {
-          //
-          //                // This avoids type erasure warnings in the generated pattern match
-          //                val pat = Bind(newTermName("erasedInst"), Typed(Ident("_"), TypeTree(erasedTpe)))
-          //                val cast = mkVal("inst", NoFlags, false, dSubType.tpe, mkAsInstanceOf(Ident("erasedInst"))(c.WeakTypeTag(dSubType.tpe)))
-          //                val inst = Ident(cast.symbol)
-          //                (pat, Some(cast), inst)
-          //              }
-          //            }
-
-          val tag = genSerialize(tagField.desc, c.literal(i).tree, target, env.copy(chkNull = false))
-          val code = genSerialize(dSubType, inst, target, env.copy(chkNull = false))
-          val body = (cast.toSeq ++ tag ++ code) :+ mkUnit
-
-          CaseDef(pat, EmptyTree, Block(body.init.toList, body.last))
-        }
-      }
-
-      Seq(mkIf(chk, Block(fields.toList,Match(source, cases))))
-    }
-
-    case RecursiveDescriptor(id, tpe, refId) => {
-      // Important: recursive types introduce re-entrant calls to serialize()
-
-      val chk = mkAnd(env.mkChkIdx(id), env.mkChkNotNull(source, tpe))
-
-      // Persist the outer record prior to recursing, since the call
-      // is going to reuse all the PactPrimitive wrappers that were 
-      // needed *before* the recursion.
-      val updTgt = Apply(Select(target, "updateBinaryRepresenation"), List())
-
-      val rec = mkVal("record" + id, NoFlags, false, typeOf[org.apache.flink.types.Record], New(TypeTree(typeOf[org.apache.flink.types.Record]), List(List())))
-      val ser = env.mkCallSerialize(refId, source, Ident("record" + id: TermName))
-
-      // Persist the new inner record after recursing, since the
-      // current call is going to reuse all the PactPrimitive
-      // wrappers that are needed *after* the recursion.
-      val updRec = Apply(Select(Ident("record" + id: TermName), "updateBinaryRepresenation"), List())
-
-      val set = env.mkSetField(id, target, Ident("record" + id: TermName))
-
-      Seq(mkIf(chk, Block(List(updTgt, rec, ser, updRec), set)))
-    }
-  }
-
-  private def genSerializeList(elem: UDTDescriptor, iter: Tree, target: Tree, env: GenEnvironment): Seq[Tree] = {
-
-    val it = mkVal("it", NoFlags, false, mkIteratorOf(elem.tpe), iter)
-
-    val loop = mkWhile(Select(Ident("it": TermName), "hasNext")) {
-
-      val item = mkVal("item", NoFlags, false, elem.tpe, Select(Ident("it": TermName), "next"))
-
-      val (stats, value) = elem match {
-
-        case PrimitiveDescriptor(_, _, _, wrapper) => (Seq(), New(TypeTree(wrapper), List(List(Ident("item": TermName)))))
-
-        case BoxedPrimitiveDescriptor(_, _, _, wrapper, _, unbox) => (Seq(), New(TypeTree(wrapper), List(List(unbox(Ident("item": TermName))))))
-        
-        case PactValueDescriptor(_, tpe) => (Seq(), Ident("item": TermName))
-
-        case ListDescriptor(id, _, iter, innerElem) => {
-          val listTpe = env.listImpls(id)
-          val list = mkVal("list" + id, NoFlags, false, listTpe, New(TypeTree(listTpe), List(List())))
-          val body = genSerializeList(innerElem, iter(Ident("item": TermName)), Ident("list" + id: TermName), env)
-          (list +: body, Ident("list" + id: TermName))
-        }
-
-        case RecursiveDescriptor(id, tpe, refId) => {
-          val rec = mkVal("record" + id, NoFlags, false, typeOf[org.apache.flink.types.Record], New(TypeTree(typeOf[org.apache.flink.types.Record]), List(List())))
-          val ser = env.mkCallSerialize(refId, Ident("item": TermName), Ident("record" + id: TermName))
-          val updRec = Apply(Select(Ident("record" + id: TermName), "updateBinaryRepresenation"), List())
-
-          (Seq(rec, ser, updRec), Ident("record" + id: TermName))
-        }
-
-        case _ => {
-          val rec = mkVal("record", NoFlags, false, typeOf[org.apache.flink.types.Record], New(TypeTree(typeOf[org.apache.flink.types.Record]), List(List())))
-          val ser = genSerialize(elem, Ident("item": TermName), Ident("record": TermName), env.copy(idxPrefix = "boxed" + elem.id, chkIndex = false, chkNull = false))
-          val upd = Apply(Select(Ident("record": TermName), "updateBinaryRepresenation"), List())
-          ((rec +: ser) :+ upd, Ident("record": TermName))
-        }
-      }
-
-      val chk = env.mkChkNotNull(Ident("item": TermName), elem.tpe)
-      val add = Apply(Select(target, "add"), List(value))
-      val addNull = Apply(Select(target, "add"), List(mkNull))
-
-      Block(List(item), mkIf(chk, mkSingle(stats :+ add), addNull))
-    }
-
-    Seq(it, loop)
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializerGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializerGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializerGen.scala
deleted file mode 100644
index 9e3a31d..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/SerializerGen.scala
+++ /dev/null
@@ -1,328 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.language.postfixOps
-import scala.reflect.macros.Context
-
-import org.apache.flink.api.scala.analysis.UDTSerializer
-
-import org.apache.flink.types.Record
-
-
-trait SerializerGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with UDTAnalyzer[C] with TreeGen[C] with SerializeMethodGen[C] with DeserializeMethodGen[C] with Loggers[C] =>
-  import c.universe._
-
-  def mkUdtSerializerClass[T: c.WeakTypeTag](name: String = "", creatorName: String = "createSerializer"): (ClassDef, Tree) = {
-    val desc = getUDTDescriptor(weakTypeOf[T])
-
-    desc match {
-      case UnsupportedDescriptor(_, _, errs) => {
-        val errorString = errs.mkString("\n")
-        c.abort(c.enclosingPosition, s"Error analyzing UDT ${weakTypeOf[T]}: $errorString")
-      }
-      case _ =>
-    }
-
-    val serName = newTypeName("UDTSerializerImpl" + name)
-    val ser = mkClass(serName, Flag.FINAL, List(weakTypeOf[UDTSerializer[T]]), {
-
-      val (listImpls, listImplTypes) = mkListImplClasses(desc)
-
-      val indexMapIter = Select(Ident("indexMap": TermName), "iterator": TermName)
-      val (fields1, inits1) = mkIndexes(desc.id, getIndexFields(desc).toList, false, indexMapIter)
-      val (fields2, inits2) = mkBoxedIndexes(desc)
-
-      val fields = fields1 ++ fields2
-      val init = inits1 ++ inits2 match {
-        case Nil => Nil
-        case inits => List(mkMethod("init", Flag.OVERRIDE | Flag.FINAL, List(), definitions.UnitTpe, Block(inits, mkUnit)))
-      }
-
-      val (wrapperFields, wrappers) = mkPactWrappers(desc, listImplTypes)
-
-      val mutableUdts = desc.flatten.toList flatMap {
-        case cc @ CaseClassDescriptor(_, _, true, _, _) => Some(cc)
-        case _ => None
-      } distinct
-
-      val mutableUdtInsts = mutableUdts map { u => mkMutableUdtInst(u) }
-
-      val helpers = listImpls ++ fields ++ wrapperFields ++ mutableUdtInsts ++ init
-      val ctor = mkMethod(nme.CONSTRUCTOR.toString(), NoFlags, List(("indexMap", typeOf[Array[Int]])), NoType, {
-        Block(List(mkSuperCall(List(Ident(newTermName("indexMap"))))), mkUnit)
-      })
-      
-//      val methods = List(ctor)// ++ List(mkGetFieldIndex(desc)) //++ mkSerialize(desc, listImplTypes) ++ mkDeserialize(desc, listImplTypes)
-      val methods = List(ctor) ++ mkSerialize(desc, listImplTypes) ++ mkDeserialize(desc, listImplTypes)
-
-      helpers ++ methods
-    })
-    
-
-    val (_, serTpe) = typeCheck(ser)
-    
-    val createSerializer = mkMethod(creatorName, Flag.OVERRIDE, List(("indexMap", typeOf[Array[Int]])), NoType, {
-      Block(List(), mkCtorCall(serTpe, List(Ident(newTermName("indexMap")))))
-    })
-    (ser, createSerializer)
-  }
-
-  private def mkListImplClass[T <: org.apache.flink.types.Value: c.WeakTypeTag]: (Tree, Type) = {
-    val listImplName = c.fresh[TypeName]("PactListImpl")
-    val tpe = weakTypeOf[org.apache.flink.types.ListValue[T]]
-
-    val listDef = mkClass(listImplName, Flag.FINAL, List(tpe), {
-      List(mkMethod(nme.CONSTRUCTOR.toString(), NoFlags, List(), NoType, Block(List(mkSuperCall()), mkUnit)))
-    })
-
-    typeCheck(listDef)
-  }
-
-  def mkListImplClasses(desc: UDTDescriptor): (List[Tree], Map[Int, Type]) = {
-    desc match {
-      case ListDescriptor(id, _, _, elem: ListDescriptor) => {
-        val (defs, tpes) = mkListImplClasses(elem)
-        val (listDef, listTpe) = mkListImplClass(c.WeakTypeTag(tpes(elem.id)))
-        (defs :+ listDef, tpes + (id -> listTpe))
-      }
-      case ListDescriptor(id, _, _, elem: PrimitiveDescriptor) => {
-        val (classDef, tpe) = mkListImplClass(c.WeakTypeTag(elem.wrapper))
-        (List(classDef), Map(id -> tpe))
-      }
-      case ListDescriptor(id, _, _, elem: BoxedPrimitiveDescriptor) => {
-        val (classDef, tpe) = mkListImplClass(c.WeakTypeTag(elem.wrapper))
-        (List(classDef), Map(id -> tpe))
-      }
-      case ListDescriptor(id, _, _, elem: PactValueDescriptor) => {
-        val (classDef, tpe) = mkListImplClass(c.WeakTypeTag(elem.tpe))
-        (List(classDef), Map(id -> tpe))
-      }
-      case ListDescriptor(id, _, _, elem) => {
-        val (classDefs, tpes) = mkListImplClasses(elem)
-        val (classDef, tpe) = mkListImplClass(c.WeakTypeTag(typeOf[org.apache.flink.types.Record]))
-        (classDefs :+ classDef, tpes + (id -> tpe))
-      }
-      case BaseClassDescriptor(_, _, getters, subTypes) => {
-        val (defs, tpes) = getters.foldLeft((List[Tree](), Map[Int, Type]())) { (result, f) =>
-          val (defs, tpes) = result
-          val (newDefs, newTpes) = mkListImplClasses(f.desc)
-          (defs ++ newDefs, tpes ++ newTpes)
-        }
-        val (subDefs, subTpes) = subTypes.foldLeft((List[Tree](), Map[Int, Type]())) { (result, s) =>
-          val (defs, tpes) = result
-          val (innerDefs, innerTpes) = mkListImplClasses(s)
-          (defs ++ innerDefs, tpes ++ innerTpes)
-        }
-        (defs ++ subDefs, tpes ++ subTpes)
-      }
-      case CaseClassDescriptor(_, _, _, _, getters) => {
-        getters.foldLeft((List[Tree](), Map[Int, Type]())) { (result, f) =>
-          val (defs, tpes) = result
-          val (newDefs, newTpes) = mkListImplClasses(f.desc)
-          (defs ++ newDefs, tpes ++ newTpes)
-        }
-      }
-      case _ => {
-        (List[Tree](), Map[Int, Type]())
-      }
-    }
-  }
-
-  private def mkIndexes(descId: Int, descFields: List[UDTDescriptor], boxed: Boolean, indexMapIter: Tree): (List[Tree], List[Tree]) = {
-
-    val prefix = (if (boxed) "boxed" else "flat") + descId
-    val iterName = prefix + "Iter"
-    val iter = mkVal(iterName, Flag.PRIVATE, true, mkIteratorOf(definitions.IntTpe), indexMapIter)
-
-    val fieldsAndInits = descFields map {
-      case d => {
-        val next = Apply(Select(Ident(iterName: TermName), "next": TermName), Nil)
-        val idxField = mkVal(prefix + "Idx" + d.id, Flag.PRIVATE, false, definitions.IntTpe, next)
-
-        (List(idxField), Nil)
-      }
-    }
-
-    val (fields, inits) = fieldsAndInits.unzip
-    (iter +: fields.flatten, inits.flatten)
-  }
-
-  protected def getIndexFields(desc: UDTDescriptor): Seq[UDTDescriptor] = desc match {
-    // Flatten product types
-    case CaseClassDescriptor(_, _, _, _, getters) => getters filterNot { _.isBaseField } flatMap { f => getIndexFields(f.desc) }
-    // TODO: Rather than laying out subclass fields sequentially, just reserve enough fields for the largest subclass.
-    // This is tricky because subclasses can contain opaque descriptors, so we don't know how many fields we need until runtime.
-    case BaseClassDescriptor(id, _, getters, subTypes) => (getters flatMap { f => getIndexFields(f.desc) }) ++ (subTypes flatMap getIndexFields)
-    case _ => Seq(desc)
-  }
-
-  private def mkBoxedIndexes(desc: UDTDescriptor): (List[Tree], List[Tree]) = {
-
-    def getBoxedDescriptors(d: UDTDescriptor): Seq[UDTDescriptor] = d match {
-      case ListDescriptor(_, _, _, elem: BaseClassDescriptor) => elem +: getBoxedDescriptors(elem)
-      case ListDescriptor(_, _, _, elem: CaseClassDescriptor) => elem +: getBoxedDescriptors(elem)
-      case ListDescriptor(_, _, _, elem) => getBoxedDescriptors(elem)
-      case CaseClassDescriptor(_, _, _, _, getters) => getters filterNot { _.isBaseField } flatMap { f => getBoxedDescriptors(f.desc) }
-      case BaseClassDescriptor(id, _, getters, subTypes) => (getters flatMap { f => getBoxedDescriptors(f.desc) }) ++ (subTypes flatMap getBoxedDescriptors)
-      case RecursiveDescriptor(_, _, refId) => desc.findById(refId).map(_.mkRoot).toSeq
-      case _ => Seq()
-    }
-
-    val fieldsAndInits = getBoxedDescriptors(desc).distinct.toList flatMap { d =>
-      // the way this is done here is a relic from the support of OpaqueDescriptors
-      // there it was not just mkOne but actual differing numbers of fields
-      // retrieved from the opaque UDT descriptors
-      getIndexFields(d).toList match {
-        case Nil => None
-        case fields => {
-          val widths = fields map {
-            case _ => mkOne
-          }
-          val sum = widths.reduce { (s, i) => Apply(Select(s, "$plus": TermName), List(i)) }
-          val range = Apply(Select(Ident("scala": TermName), "Range": TermName), List(mkZero, sum))
-          Some(mkIndexes(d.id, fields, true, Select(range, "iterator": TermName)))
-        }
-      }
-    }
-
-    val (fields, inits) = fieldsAndInits.unzip
-    (fields.flatten, inits.flatten)
-  }
-
-  private def mkPactWrappers(desc: UDTDescriptor, listImpls: Map[Int, Type]): (List[Tree], List[(Int, Type)]) = {
-
-    def getFieldTypes(desc: UDTDescriptor): Seq[(Int, Type)] = desc match {
-      case PrimitiveDescriptor(id, _, _, wrapper) => Seq((id, wrapper))
-      case BoxedPrimitiveDescriptor(id, _, _, wrapper, _, _) => Seq((id, wrapper))
-      case d @ ListDescriptor(id, _, _, elem) => {
-        val listField = (id, listImpls(id))
-        val elemFields = d.getInnermostElem match {
-          case elem: CaseClassDescriptor => getFieldTypes(elem)
-          case elem: BaseClassDescriptor => getFieldTypes(elem)
-          case _ => Seq()
-        }
-        listField +: elemFields
-      }
-      case CaseClassDescriptor(_, _, _, _, getters) => getters filterNot { _.isBaseField } flatMap { f => getFieldTypes(f.desc) }
-      case BaseClassDescriptor(_, _, getters, subTypes) => (getters flatMap { f => getFieldTypes(f.desc) }) ++ (subTypes flatMap getFieldTypes)
-      case _ => Seq()
-    }
-
-    getFieldTypes(desc) toList match {
-      case Nil => (Nil, Nil)
-      case types =>
-        val fields = types map { case (id, tpe) => mkVar("w" + id, Flag.PRIVATE, true, tpe, mkCtorCall(tpe, List())) }
-        (fields, types)
-    }
-  }
-
-  private def mkMutableUdtInst(desc: CaseClassDescriptor): Tree = {
-    val args = desc.getters map {
-      case FieldAccessor(_, _, fTpe, _, _) => {
-        mkDefault(fTpe)
-      }
-    }
-
-    val ctor = mkCtorCall(desc.tpe, args.toList)
-    mkVar("mutableUdtInst" + desc.id, Flag.PRIVATE, true, desc.tpe, ctor)
-  }
-
-  private def mkGetFieldIndex(desc: UDTDescriptor): Tree = {
-
-    val env = GenEnvironment(Map(), "flat" + desc.id, false, true, true, true)
-
-    def mkCases(desc: UDTDescriptor, path: Seq[String]): Seq[(Seq[String], Tree)] = desc match {
-
-      case PrimitiveDescriptor(id, _, _, _) => Seq((path, mkList(List(env.mkSelectIdx(id)))))
-      case BoxedPrimitiveDescriptor(id, _, _, _, _, _) => Seq((path, mkList(List(env.mkSelectIdx(id)))))
-
-      case BaseClassDescriptor(_, _, Seq(tag, getters @ _*), _) => {
-        val tagCase = Seq((path :+ "getClass", mkList(List(env.mkSelectIdx(tag.desc.id)))))
-        val fieldCases = getters flatMap { f => mkCases(f.desc, path :+ f.getter.name.toString) }
-        tagCase ++ fieldCases
-      }
-
-      case CaseClassDescriptor(_, _, _, _, getters) => {
-        def fieldCases = getters flatMap { f => mkCases(f.desc, path :+ f.getter.name.toString) }
-        val allFieldsCase = desc match {
-          case _ if desc.isPrimitiveProduct => {
-            val nonRest = fieldCases filter { case (p, _) => p.size == path.size + 1 } map { _._2 }
-            Seq((path, nonRest.reduceLeft((z, f) => Apply(Select(z, "$plus$plus"), List(f)))))
-          }
-          case _ => Seq()
-        }
-        allFieldsCase ++ fieldCases
-      }
-      case _ => Seq()
-    }
-
-    def mkPat(path: Seq[String]): Tree = {
-
-      val seqUnapply = TypeApply(mkSelect("scala", "collection", "Seq", "unapplySeq"), List(TypeTree(typeOf[String])))
-      val fun = Apply(seqUnapply, List(Ident(nme.WILDCARD)))
-      
-      val args = path map {
-        case null => Bind(newTermName("rest"), Star(Ident(newTermName("_"))))
-        case s => Literal(Constant(s))
-      }
-
-      UnApply(fun, args.toList)
-    }
-
-    mkMethod("getFieldIndex", Flag.FINAL, List(("selection", mkSeqOf(typeOf[String]))), mkListOf(typeOf[Int]), {
-//    mkMethod("getFieldIndex", Flag.OVERRIDE | Flag.FINAL, List(("selection", mkSeqOf(typeOf[String]))), mkListOf(typeOf[Int]), {
-      val cases = mkCases(desc, Seq()) map { case (path, idxs) => CaseDef(mkPat(path), EmptyTree, idxs) }
-//      val errCase = CaseDef(Ident("_"), EmptyTree, Apply(Ident(newTermName("println")), List(Ident("selection"))))
-      val errCase = CaseDef(Ident("_"), EmptyTree, (reify {throw new RuntimeException("Invalid selection")}).tree )
-//      Match(Ident("selection"), (cases :+ errCase).toList)
-      Match(Ident("selection"), List(errCase))
-    })
-  }
-
-  protected case class GenEnvironment(listImpls: Map[Int, Type], idxPrefix: String, reentrant: Boolean, allowRecycling: Boolean, chkIndex: Boolean, chkNull: Boolean) {
-    private def isNullable(tpe: Type) = typeOf[Null] <:< tpe && tpe <:< typeOf[AnyRef]
-
-    def mkChkNotNull(source: Tree, tpe: Type): Tree = if (isNullable(tpe) && chkNull) Apply(Select(source, "$bang$eq": TermName), List(mkNull)) else EmptyTree
-    def mkChkIdx(fieldId: Int): Tree = if (chkIndex) Apply(Select(mkSelectIdx(fieldId), "$greater$eq": TermName), List(mkZero)) else EmptyTree
-
-    def mkSelectIdx(fieldId: Int): Tree = Ident(newTermName(idxPrefix + "Idx" + fieldId))
-    def mkSelectSerializer(fieldId: Int): Tree = Ident(newTermName(idxPrefix + "Ser" + fieldId))
-    def mkSelectWrapper(fieldId: Int): Tree = Ident(newTermName("w" + fieldId))
-    def mkSelectMutableUdtInst(udtId: Int): Tree = Ident(newTermName("mutableUdtInst" + udtId))
-
-    def mkCallSetMutableField(udtId: Int, setter: Symbol, source: Tree): Tree = Apply(Select(mkSelectMutableUdtInst(udtId), setter), List(source))
-    def mkCallSerialize(refId: Int, source: Tree, target: Tree): Tree = Apply(Ident(newTermName("serialize" + refId)), List(source, target))
-    def mkCallDeserialize(refId: Int, source: Tree): Tree = Apply(Ident(newTermName("deserialize" + refId)), List(source))
-
-    def mkSetField(fieldId: Int, record: Tree): Tree = mkSetField(fieldId, record, mkSelectWrapper(fieldId))
-    def mkSetField(fieldId: Int, record: Tree, wrapper: Tree): Tree = Apply(Select(record, "setField": TermName), List(mkSelectIdx(fieldId), wrapper))
-    def mkGetField(fieldId: Int, record: Tree, tpe: Type): Tree = Apply(Select(record, "getField": TermName), List(mkSelectIdx(fieldId), Literal(Constant(tpe))))
-    def mkGetFieldInto(fieldId: Int, record: Tree): Tree = mkGetFieldInto(fieldId, record, mkSelectWrapper(fieldId))
-    def mkGetFieldInto(fieldId: Int, record: Tree, wrapper: Tree): Tree = Apply(Select(record, "getFieldInto": TermName), List(mkSelectIdx(fieldId), wrapper))
-
-    def mkSetValue(fieldId: Int, value: Tree): Tree = mkSetValue(mkSelectWrapper(fieldId), value)
-    def mkSetValue(wrapper: Tree, value: Tree): Tree = Apply(Select(wrapper, "setValue": TermName), List(value))
-    def mkGetValue(fieldId: Int): Tree = mkGetValue(mkSelectWrapper(fieldId))
-    def mkGetValue(wrapper: Tree): Tree = Apply(Select(wrapper, "getValue": TermName), List())
-
-    def mkNotIsNull(fieldId: Int, record: Tree): Tree = Select(Apply(Select(record, "isNull": TermName), List(mkSelectIdx(fieldId))), "unary_$bang": TermName)
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
index 29bf6ed..89454d5 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
@@ -23,11 +23,10 @@ import scala.language.implicitConversions
 
 import scala.reflect.macros.Context
 
-trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with Loggers[C] =>
+private[flink] trait TreeGen[C <: Context] { this: MacroContextHolder[C] with TypeDescriptors[C] =>
   import c.universe._
 
     def mkDefault(tpe: Type): Tree = {
-      import definitions._
       tpe match {
         case definitions.BooleanTpe => Literal(Constant(false))
         case definitions.ByteTpe    => Literal(Constant(0: Byte))
@@ -47,7 +46,8 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
     def mkZero = reify( 0 ).tree
     def mkOne = reify( 1 ).tree
 
-    def mkAsInstanceOf[T: c.WeakTypeTag](source: Tree): Tree = reify(c.Expr(source).splice.asInstanceOf[T]).tree
+    def mkAsInstanceOf[T: c.WeakTypeTag](source: Tree): Tree =
+      reify(c.Expr(source).splice.asInstanceOf[T]).tree
 
     def maybeMkAsInstanceOf[S: c.WeakTypeTag, T: c.WeakTypeTag](source: Tree): Tree = {
       if (weakTypeOf[S] <:< weakTypeOf[T])
@@ -57,14 +57,25 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
     }
 
 //    def mkIdent(target: Symbol): Tree = Ident(target) setType target.tpe
-    def mkSelect(rootModule: String, path: String*): Tree = mkSelect(Ident(newTermName(rootModule)), path: _*)
-    def mkSelect(source: Tree, path: String*): Tree = path.foldLeft(source) { (ret, item) => Select(ret, newTermName(item)) }
-    def mkSelectSyms(source: Tree, path: Symbol*): Tree = path.foldLeft(source) { (ret, item) => Select(ret, item) }
+    def mkSelect(rootModule: String, path: String*): Tree =
+      mkSelect(Ident(newTermName(rootModule)), path: _*)
+
+    def mkSelect(source: Tree, path: String*): Tree =
+      path.foldLeft(source) { (ret, item) => Select(ret, newTermName(item)) }
+
+    def mkSelectSyms(source: Tree, path: Symbol*): Tree =
+      path.foldLeft(source) { (ret, item) => Select(ret, item) }
     
     def mkCall(root: Tree, path: String*)(args: List[Tree]) = Apply(mkSelect(root, path: _*), args)
 
-    def mkSeq(items: List[Tree]): Tree = Apply(mkSelect("scala", "collection", "Seq", "apply"), items)
-    def mkList(items: List[Tree]): Tree = Apply(mkSelect("scala", "collection", "immutable", "List", "apply"), items)
+    def mkSeq(items: List[Tree]): Tree =
+      Apply(mkSelect("scala", "collection", "Seq", "apply"), items)
+
+    def mkList(items: List[Tree]): Tree =
+      Apply(mkSelect("scala", "collection", "immutable", "List", "apply"), items)
+
+    def mkMap(items: List[Tree]): Tree =
+      Apply(mkSelect("scala", "collection", "immutable", "Map", "apply"), items)
 
     def mkVal(name: String, flags: FlagSet, transient: Boolean, valTpe: Type, value: Tree): Tree = {
       ValDef(Modifiers(flags), newTermName(name), TypeTree(valTpe), value)
@@ -81,7 +92,11 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
       List(valDef, defDef)
     }
 
-    def mkVarAndLazyGetter(name: String, flags: FlagSet, valTpe: Type, value: Tree): (Tree, Tree) = {
+    def mkVarAndLazyGetter(
+        name: String,
+        flags: FlagSet,
+        valTpe: Type,
+        value: Tree): (Tree, Tree) = {
       val fieldName = name + " "
       val field = mkVar(fieldName, NoFlags, false, valTpe, mkNull)
       val fieldSel = Ident(newTermName(fieldName))
@@ -117,20 +132,35 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
       }
     }
 
-    def mkMethod(name: String, flags: FlagSet, args: List[(String, Type)], ret: Type, impl: Tree): Tree = {
-      val valParams = args map { case (name, tpe) => ValDef(Modifiers(Flag.PARAM), newTermName(name), TypeTree(tpe), EmptyTree) }
+    def mkMethod(
+        name: String,
+        flags: FlagSet,
+        args: List[(String, Type)],
+        ret: Type,
+        impl: Tree): Tree = {
+      val valParams = args map { case (name, tpe) =>
+        ValDef(Modifiers(Flag.PARAM), newTermName(name), TypeTree(tpe), EmptyTree)
+      }
       DefDef(Modifiers(flags), newTermName(name), Nil, List(valParams), TypeTree(ret), impl)
     }
 
-    def mkClass(name: TypeName, flags: FlagSet, parents: List[Type], members: List[Tree]): ClassDef = {
+    def mkClass(
+        name: TypeName,
+        flags: FlagSet,
+        parents: List[Type],
+        members: List[Tree]): ClassDef = {
       val parentTypeTrees = parents map { TypeTree(_) }
       val selfType = ValDef(Modifiers(), nme.WILDCARD, TypeTree(NoType), EmptyTree)
       ClassDef(Modifiers(flags), name, Nil, Template(parentTypeTrees, selfType, members))
     }
 
-    def mkThrow(tpe: Type, msg: Tree): Tree = Throw(Apply(Select(New(TypeTree(tpe)), nme.CONSTRUCTOR), List(msg)))
+    def mkThrow(tpe: Type, msg: Tree): Tree =
+      Throw(Apply(Select(New(TypeTree(tpe)), nme.CONSTRUCTOR), List(msg)))
+
 //    def mkThrow(tpe: Type, msg: Tree): Tree = Throw(New(TypeTree(tpe)), List(List(msg))))
+
     def mkThrow(tpe: Type, msg: String): Tree = mkThrow(tpe, c.literal(msg).tree)
+
     def mkThrow(msg: String): Tree = mkThrow(typeOf[java.lang.RuntimeException], msg)
 
     implicit def tree2Ops[T <: Tree](tree: T) = new {
@@ -165,17 +195,24 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
     }
     
     def mkBuilderOf(elemTpe: Type, listTpe: Type) = {
-      def makeIt[ElemTpe: c.WeakTypeTag, ListTpe: c.WeakTypeTag] = weakTypeOf[scala.collection.mutable.Builder[ElemTpe, ListTpe]]
+      def makeIt[ElemTpe: c.WeakTypeTag, ListTpe: c.WeakTypeTag] =
+        weakTypeOf[scala.collection.mutable.Builder[ElemTpe, ListTpe]]
+
       makeIt(c.WeakTypeTag(elemTpe), c.WeakTypeTag(listTpe))
     }
     
     def mkCanBuildFromOf(fromTpe: Type, elemTpe: Type, toTpe: Type) = {
-      def makeIt[From: c.WeakTypeTag, Elem: c.WeakTypeTag, To: c.WeakTypeTag] = weakTypeOf[scala.collection.generic.CanBuildFrom[From, Elem, To]]
+      def makeIt[From: c.WeakTypeTag, Elem: c.WeakTypeTag, To: c.WeakTypeTag] =
+        weakTypeOf[scala.collection.generic.CanBuildFrom[From, Elem, To]]
+
       makeIt(c.WeakTypeTag(fromTpe), c.WeakTypeTag(elemTpe), c.WeakTypeTag(toTpe))
     }
     
-    def mkCtorCall(tpe: Type, args: List[Tree]) = Apply(Select(New(TypeTree(tpe)), nme.CONSTRUCTOR), args)
-    def mkSuperCall(args: List[Tree] = List()) = Apply(Select(Super(This(tpnme.EMPTY), tpnme.EMPTY), nme.CONSTRUCTOR), args)
+    def mkCtorCall(tpe: Type, args: List[Tree]) =
+      Apply(Select(New(TypeTree(tpe)), nme.CONSTRUCTOR), args)
+
+    def mkSuperCall(args: List[Tree] = List()) =
+      Apply(Select(Super(This(tpnme.EMPTY), tpnme.EMPTY), nme.CONSTRUCTOR), args)
 
     def mkWhile(cond: Tree)(body: Tree): Tree = {
       val lblName = c.fresh[TermName]("while")
@@ -202,7 +239,8 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
     def extractOneInputUdf(fun: Tree) = {
       val (paramName, udfBody) = fun match {
         case Function(List(param), body) => (param.name.toString, body)
-        case _ => c.abort(c.enclosingPosition, "Could not extract user defined function, got: " + show(fun))
+        case _ =>
+          c.abort(c.enclosingPosition, "Could not extract user defined function, got: " + show(fun))
       }
       val uncheckedUdfBody = c.resetAllAttrs(udfBody)
       (paramName, uncheckedUdfBody)
@@ -210,8 +248,10 @@ trait TreeGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C]
     
     def extractTwoInputUdf(fun: Tree) = {
       val (param1Name, param2Name, udfBody) = fun match {
-        case Function(List(param1, param2), body) => (param1.name.toString, param2.name.toString, body)
-        case _ => c.abort(c.enclosingPosition, "Could not extract user defined function, got: " + show(fun))
+        case Function(List(param1, param2), body) =>
+          (param1.name.toString, param2.name.toString, body)
+        case _ =>
+          c.abort(c.enclosingPosition, "Could not extract user defined function, got: " + show(fun))
       }
       val uncheckedUdfBody = c.resetAllAttrs(udfBody)
       (param1Name, param2Name, uncheckedUdfBody)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
new file mode 100644
index 0000000..7b1675d
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
@@ -0,0 +1,382 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.api.scala.codegen
+
+import scala.Option.option2Iterable
+import scala.collection.GenTraversableOnce
+import scala.collection.mutable
+import scala.reflect.macros.Context
+import scala.util.DynamicVariable
+
+import org.apache.flink.types.BooleanValue
+import org.apache.flink.types.ByteValue
+import org.apache.flink.types.CharValue
+import org.apache.flink.types.DoubleValue
+import org.apache.flink.types.FloatValue
+import org.apache.flink.types.IntValue
+import org.apache.flink.types.StringValue
+import org.apache.flink.types.LongValue
+import org.apache.flink.types.ShortValue
+
+
+private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
+  with TypeDescriptors[C] =>
+
+  import c.universe._
+
+  // This value is controlled by the udtRecycling compiler option
+  var enableMutableUDTs = false
+
+  private val mutableTypes = mutable.Set[Type]()
+
+  def getUDTDescriptor(tpe: Type): UDTDescriptor = new UDTAnalyzerInstance().analyze(tpe)
+
+  private def typeArgs(tpe: Type) = tpe match { case TypeRef(_, _, args) => args }
+
+  private class UDTAnalyzerInstance {
+
+    private val cache = new UDTAnalyzerCache()
+
+    def analyze(tpe: Type): UDTDescriptor = {
+
+      cache.getOrElseUpdate(tpe) { id =>
+        tpe match {
+          case PrimitiveType(default, wrapper) => PrimitiveDescriptor(id, tpe, default, wrapper)
+          case BoxedPrimitiveType(default, wrapper, box, unbox) =>
+            BoxedPrimitiveDescriptor(id, tpe, default, wrapper, box, unbox)
+          case ListType(elemTpe, iter) => analyzeList(id, tpe, elemTpe, iter)
+          case CaseClassType() => analyzeCaseClass(id, tpe)
+          case BaseClassType() => analyzeClassHierarchy(id, tpe)
+          case ValueType() => ValueDescriptor(id, tpe)
+          case WritableType() => WritableDescriptor(id, tpe)
+          case _ => GenericClassDescriptor(id, tpe)
+        }
+      }
+    }
+
+    private def analyzeList(
+        id: Int,
+        tpe: Type,
+        elemTpe: Type,
+        iter: Tree => Tree): UDTDescriptor = analyze(elemTpe) match {
+      case UnsupportedDescriptor(_, _, errs) => UnsupportedDescriptor(id, tpe, errs)
+      case desc => ListDescriptor(id, tpe, iter, desc)
+    }
+
+    private def analyzeClassHierarchy(id: Int, tpe: Type): UDTDescriptor = {
+
+      val tagField = {
+        val (intTpe, intDefault, intWrapper) = PrimitiveType.intPrimitive
+        FieldAccessor(
+          NoSymbol,
+          NoSymbol,
+          NullaryMethodType(intTpe),
+          isBaseField = true,
+          PrimitiveDescriptor(cache.newId, intTpe, intDefault, intWrapper))
+      }
+      
+      val subTypes = tpe.typeSymbol.asClass.knownDirectSubclasses.toList flatMap { d =>
+
+        val dTpe =
+          {
+            val tArgs = (tpe.typeSymbol.asClass.typeParams, typeArgs(tpe)).zipped.toMap
+            val dArgs = d.asClass.typeParams map { dp =>
+              val tArg = tArgs.keySet.find { tp =>
+                dp == tp.typeSignature.asSeenFrom(d.typeSignature, tpe.typeSymbol).typeSymbol
+              }
+              tArg map { tArgs(_) } getOrElse dp.typeSignature
+            }
+
+            appliedType(d.asType.toType, dArgs)
+          }
+
+        if (dTpe <:< tpe)
+          Some(analyze(dTpe))
+        else
+          None
+      }
+
+      val errors = subTypes flatMap { _.findByType[UnsupportedDescriptor] }
+
+      errors match {
+        case _ :: _ =>
+          val errorMessage = errors flatMap {
+            case UnsupportedDescriptor(_, subType, errs) =>
+              errs map { err => "Subtype " + subType + " - " + err }
+          }
+          UnsupportedDescriptor(id, tpe, errorMessage)
+
+        case Nil if subTypes.isEmpty =>
+          UnsupportedDescriptor(id, tpe, Seq("No instantiable subtypes found for base class"))
+        case Nil =>
+          val (tParams, _) = tpe.typeSymbol.asClass.typeParams.zip(typeArgs(tpe)).unzip
+          val baseMembers =
+            tpe.members filter { f => f.isMethod } filter { f => f.asMethod.isSetter } map {
+            f => (f, f.asMethod.setter, f.asMethod.returnType)
+          }
+
+          val subMembers = subTypes map {
+            case BaseClassDescriptor(_, _, getters, _) => getters
+            case CaseClassDescriptor(_, _, _, _, getters) => getters
+            case _ => Seq()
+          }
+
+          val baseFields = baseMembers flatMap {
+            case (bGetter, bSetter, bTpe) =>
+              val accessors = subMembers map {
+                _ find { sf =>
+                  sf.getter.name == bGetter.name &&
+                    sf.tpe.termSymbol.asMethod.returnType <:< bTpe.termSymbol.asMethod.returnType
+                }
+              }
+              accessors.forall { _.isDefined } match {
+                case true =>
+                  Some(
+                    FieldAccessor(
+                      bGetter, bSetter, bTpe, isBaseField = true, analyze(bTpe.termSymbol.asMethod.returnType)))
+                case false => None
+              }
+          }
+
+          def wireBaseFields(desc: UDTDescriptor): UDTDescriptor = {
+
+            def updateField(field: FieldAccessor) = {
+              baseFields find { bf => bf.getter.name == field.getter.name } match {
+                case Some(FieldAccessor(_, _, _, _, fieldDesc)) =>
+                  field.copy(isBaseField = true, desc = fieldDesc)
+                case None => field
+              }
+            }
+
+            desc match {
+              case desc @ BaseClassDescriptor(_, _, getters, baseSubTypes) =>
+                desc.copy(getters = getters map updateField, subTypes = baseSubTypes map wireBaseFields)
+              case desc @ CaseClassDescriptor(_, _, _, _, getters) =>
+                desc.copy(getters = getters map updateField)
+              case _ => desc
+            }
+          }
+
+          BaseClassDescriptor(id, tpe, tagField +: baseFields.toSeq, subTypes map wireBaseFields)
+      }
+
+    }
+
+    private def analyzeCaseClass(id: Int, tpe: Type): UDTDescriptor = {
+
+      tpe.baseClasses exists { bc => !(bc == tpe.typeSymbol) && bc.asClass.isCaseClass } match {
+
+        case true =>
+          UnsupportedDescriptor(id, tpe, Seq("Case-to-case inheritance is not supported."))
+
+        case false =>
+
+          val ctors = tpe.declarations collect {
+            case m: MethodSymbol if m.isPrimaryConstructor => m
+          }
+
+          ctors match {
+            case c1 :: c2 :: _ =>
+              UnsupportedDescriptor(
+                id,
+                tpe,
+                Seq("Multiple constructors found, this is not supported."))
+            case ctor :: Nil =>
+              val caseFields = ctor.paramss.flatten.map {
+                sym =>
+                  {
+                    val methodSym = tpe.member(sym.name).asMethod
+                    val getter = methodSym.getter
+                    val setter = methodSym.setter
+                    val returnType = methodSym.returnType.asSeenFrom(tpe, tpe.typeSymbol)
+                    (getter, setter, returnType)
+                  }
+              }
+              val fields = caseFields map {
+                case (fgetter, fsetter, fTpe) =>
+                  FieldAccessor(fgetter, fsetter, fTpe, isBaseField = false, analyze(fTpe))
+              }
+              val mutable = enableMutableUDTs && (fields forall { f => f.setter != NoSymbol })
+              if (mutable) {
+                mutableTypes.add(tpe)
+              }
+              fields filter { _.desc.isInstanceOf[UnsupportedDescriptor] } match {
+                case errs @ _ :: _ =>
+                  val msgs = errs flatMap { f =>
+                    (f: @unchecked) match {
+                      case FieldAccessor(fgetter, _, _, _, UnsupportedDescriptor(_, fTpe, errors)) =>
+                        errors map { err => "Field " + fgetter.name + ": " + fTpe + " - " + err }
+                    }
+                  }
+                  UnsupportedDescriptor(id, tpe, msgs)
+
+                case Nil => CaseClassDescriptor(id, tpe, mutable, ctor, fields.toSeq)
+              }
+          }
+      }
+    }
+
+    private object PrimitiveType {
+      def intPrimitive: (Type, Literal, Type) = {
+        val (d, w) = primitives(definitions.IntClass)
+        (definitions.IntTpe, d, w)
+      }
+
+      def unapply(tpe: Type): Option[(Literal, Type)] = primitives.get(tpe.typeSymbol)
+    }
+
+    private object BoxedPrimitiveType {
+      def unapply(tpe: Type): Option[(Literal, Type, Tree => Tree, Tree => Tree)] =
+        boxedPrimitives.get(tpe.typeSymbol)
+    }
+
+    private object ListType {
+
+      def unapply(tpe: Type): Option[(Type, Tree => Tree)] = tpe match {
+
+        case ArrayType(elemTpe) =>
+          val iter = { source: Tree => 
+            Select(source, newTermName("iterator"))
+          }
+          Some(elemTpe, iter)
+
+        case TraversableType(elemTpe) =>
+          val iter = { source: Tree => Select(source, newTermName("toIterator")) }
+          Some(elemTpe, iter)
+
+        case _ => None
+      }
+
+      private object ArrayType {
+        def unapply(tpe: Type): Option[Type] = tpe match {
+          case TypeRef(_, _, elemTpe :: Nil) if tpe <:< typeOf[Array[_]] => Some(elemTpe)
+          case _ => None
+        }
+      }
+
+      private object TraversableType {
+        def unapply(tpe: Type): Option[Type] = tpe match {
+          case _ if tpe <:< typeOf[GenTraversableOnce[_]] =>
+            // val abstrElemTpe = genTraversableOnceClass.typeConstructor.typeParams.head.tpe
+            // val elemTpe = abstrElemTpe.asSeenFrom(tpe, genTraversableOnceClass)
+            // Some(elemTpe)
+            // TODO make sure this works as it should
+            tpe match {
+              case TypeRef(_, _, elemTpe :: Nil) => Some(elemTpe.asSeenFrom(tpe, tpe.typeSymbol))
+            }
+
+          case _ => None
+        }
+      }
+    }
+
+    private object CaseClassType {
+      def unapply(tpe: Type): Boolean = tpe.typeSymbol.asClass.isCaseClass
+    }
+
+    private object BaseClassType {
+      def unapply(tpe: Type): Boolean =
+        tpe.typeSymbol.asClass.isAbstractClass && tpe.typeSymbol.asClass.isSealed
+    }
+    
+    private object ValueType {
+      def unapply(tpe: Type): Boolean =
+        tpe.typeSymbol.asClass.baseClasses exists {
+          s => s.fullName == "org.apache.flink.types.Value"
+        }
+    }
+
+    private object WritableType {
+      def unapply(tpe: Type): Boolean =
+        tpe.typeSymbol.asClass.baseClasses exists {
+          s => s.fullName == "org.apache.hadoop.io.Writable"
+        }
+    }
+
+    private class UDTAnalyzerCache {
+
+      private val caches = new DynamicVariable[Map[Type, RecursiveDescriptor]](Map())
+      private val idGen = new Counter
+
+      def newId = idGen.next
+
+      def getOrElseUpdate(tpe: Type)(orElse: Int => UDTDescriptor): UDTDescriptor = {
+
+        val id = idGen.next
+        val cache = caches.value
+
+        cache.get(tpe) map { _.copy(id = id) } getOrElse {
+          val ref = RecursiveDescriptor(id, tpe, id)
+          caches.withValue(cache + (tpe -> ref)) {
+            orElse(id)
+          }
+        }
+      }
+    }
+  }
+
+  lazy val primitives = Map[Symbol, (Literal, Type)](
+    definitions.BooleanClass -> (Literal(Constant(false)), typeOf[BooleanValue]),
+    definitions.ByteClass -> (Literal(Constant(0: Byte)), typeOf[ByteValue]),
+    definitions.CharClass -> (Literal(Constant(0: Char)), typeOf[CharValue]),
+    definitions.DoubleClass -> (Literal(Constant(0: Double)), typeOf[DoubleValue]),
+    definitions.FloatClass -> (Literal(Constant(0: Float)), typeOf[FloatValue]),
+    definitions.IntClass -> (Literal(Constant(0: Int)), typeOf[IntValue]),
+    definitions.LongClass -> (Literal(Constant(0: Long)), typeOf[LongValue]),
+    definitions.ShortClass -> (Literal(Constant(0: Short)), typeOf[ShortValue]),
+    definitions.StringClass -> (Literal(Constant(null: String)), typeOf[StringValue]))
+
+  lazy val boxedPrimitives = {
+
+    def getBoxInfo(prim: Symbol, primName: String, boxName: String) = {
+      val (default, wrapper) = primitives(prim)
+      val box = { t: Tree => 
+        Apply(
+          Select(
+            Select(Ident(newTermName("scala")), newTermName("Predef")),
+            newTermName(primName + "2" + boxName)),
+          List(t))
+      }
+      val unbox = { t: Tree =>
+        Apply(
+          Select(
+            Select(Ident(newTermName("scala")), newTermName("Predef")),
+            newTermName(boxName + "2" + primName)),
+          List(t))
+      }
+      (default, wrapper, box, unbox)
+    }
+
+    Map(
+      typeOf[java.lang.Boolean].typeSymbol ->
+        getBoxInfo(definitions.BooleanClass, "boolean", "Boolean"),
+      typeOf[java.lang.Byte].typeSymbol -> getBoxInfo(definitions.ByteClass, "byte", "Byte"),
+      typeOf[java.lang.Character].typeSymbol ->
+        getBoxInfo(definitions.CharClass, "char", "Character"),
+      typeOf[java.lang.Double].typeSymbol ->
+        getBoxInfo(definitions.DoubleClass, "double", "Double"),
+      typeOf[java.lang.Float].typeSymbol -> getBoxInfo(definitions.FloatClass, "float", "Float"),
+      typeOf[java.lang.Integer].typeSymbol -> getBoxInfo(definitions.IntClass, "int", "Integer"),
+      typeOf[java.lang.Long].typeSymbol -> getBoxInfo(definitions.LongClass, "long", "Long"),
+      typeOf[java.lang.Short].typeSymbol -> getBoxInfo(definitions.ShortClass, "short", "Short"))
+  }
+
+}
+


[24/60] Renamed java examples package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesBasic.java
new file mode 100644
index 0000000..c3cbc6d
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesBasic.java
@@ -0,0 +1,230 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.graph;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesData;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Edge;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Triad;
+
+/**
+ * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
+ * A triangle consists of three edges that connect three vertices with each other.
+ * 
+ * <p>
+ * The algorithm works as follows: 
+ * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
+ * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
+ * that closes the triangle.
+ *  
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Edges are represented as pairs for vertex IDs which are separated by space 
+ * characters. Edges are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
+ * that include a triangle
+ * </ul>
+ * <pre>
+ *     (1)
+ *     /  \
+ *   (2)-(12)
+ * </pre>
+ * 
+ * Usage: <code>EnumTriangleBasic &lt;edge path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link EnumTrianglesData}. 
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Custom Java objects which extend Tuple
+ * <li>Group Sorting
+ * </ul>
+ * 
+ */
+@SuppressWarnings("serial")
+public class EnumTrianglesBasic {
+
+	static boolean fileOutput = false;
+	static String edgePath = null;
+	static String outputPath = null;
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		// set up execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+	
+		// read input data
+		DataSet<Edge> edges = getEdgeDataSet(env);
+		
+		// project edges by vertex id
+		DataSet<Edge> edgesById = edges
+				.map(new EdgeByIdProjector());
+		
+		DataSet<Triad> triangles = edgesById
+				// build triads
+				.groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder())
+				// filter triads
+				.join(edgesById).where(Triad.V2, Triad.V3).equalTo(Edge.V1, Edge.V2).with(new TriadFilter());
+
+		// emit result
+		if(fileOutput) {
+			triangles.writeAsCsv(outputPath, "\n", ",");
+		} else {
+			triangles.print();
+		}
+
+		// execute program
+		env.execute("Basic Triangle Enumeration Example");
+
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/** Converts a Tuple2 into an Edge */
+	public static class TupleEdgeConverter implements MapFunction<Tuple2<Integer, Integer>, Edge> {
+		private final Edge outEdge = new Edge();
+		
+		@Override
+		public Edge map(Tuple2<Integer, Integer> t) throws Exception {
+			outEdge.copyVerticesFromTuple2(t);
+			return outEdge;
+		}
+	}
+	
+	/** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */
+	private static class EdgeByIdProjector implements MapFunction<Edge, Edge> {
+	
+		@Override
+		public Edge map(Edge inEdge) throws Exception {
+			
+			// flip vertices if necessary
+			if(inEdge.getFirstVertex() > inEdge.getSecondVertex()) {
+				inEdge.flipVertices();
+			}
+			
+			return inEdge;
+		}
+	}
+	
+	/**
+	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
+	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
+	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
+	 */
+	private static class TriadBuilder implements GroupReduceFunction<Edge, Triad> {
+		private final List<Integer> vertices = new ArrayList<Integer>();
+		private final Triad outTriad = new Triad();
+		
+		@Override
+		public void reduce(Iterable<Edge> edgesIter, Collector<Triad> out) throws Exception {
+			
+			final Iterator<Edge> edges = edgesIter.iterator();
+			
+			// clear vertex list
+			vertices.clear();
+			
+			// read first edge
+			Edge firstEdge = edges.next();
+			outTriad.setFirstVertex(firstEdge.getFirstVertex());
+			vertices.add(firstEdge.getSecondVertex());
+			
+			// build and emit triads
+			while (edges.hasNext()) {
+				Integer higherVertexId = edges.next().getSecondVertex();
+				
+				// combine vertex with all previously read vertices
+				for (Integer lowerVertexId : vertices) {
+					outTriad.setSecondVertex(lowerVertexId);
+					outTriad.setThirdVertex(higherVertexId);
+					out.collect(outTriad);
+				}
+				vertices.add(higherVertexId);
+			}
+		}
+	}
+	
+	/** Filters triads (three vertices connected by two edges) without a closing third edge. */
+	private static class TriadFilter implements JoinFunction<Triad, Edge, Triad> {
+		
+		@Override
+		public Triad join(Triad triad, Edge edge) throws Exception {
+			return triad;
+		}
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean parseParameters(String[] args) {
+	
+		if(args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(args.length == 2) {
+				edgePath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing Enum Triangles Basic example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>");
+		}
+		return true;
+	}
+	
+	private static DataSet<Edge> getEdgeDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return env.readCsvFile(edgePath)
+						.fieldDelimiter(' ')
+						.includeFields(true, true)
+						.types(Integer.class, Integer.class)
+						.map(new TupleEdgeConverter());
+		} else {
+			return EnumTrianglesData.getDefaultEdgeDataSet(env);
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesOpt.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesOpt.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesOpt.java
new file mode 100644
index 0000000..9dcb168
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/EnumTrianglesOpt.java
@@ -0,0 +1,354 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.graph;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesData;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Edge;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.EdgeWithDegrees;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Triad;
+
+/**
+ * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
+ * A triangle consists of three edges that connect three vertices with each other.
+ * 
+ * <p>
+ * The basic algorithm works as follows: 
+ * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
+ * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
+ * that closes the triangle.
+ * 
+ * <p>
+ * For a group of <i>n</i> edges that share a common vertex, the number of built triads is quadratic <i>((n*(n-1))/2)</i>.
+ * Therefore, an optimization of the algorithm is to group edges on the vertex with the smaller output degree to 
+ * reduce the number of triads. 
+ * This implementation extends the basic algorithm by computing output degrees of edge vertices and 
+ * grouping on edges on the vertex with the smaller degree.
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Edges are represented as pairs for vertex IDs which are separated by space 
+ * characters. Edges are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
+ * that include a triangle
+ * </ul>
+ * <pre>
+ *     (1)
+ *     /  \
+ *   (2)-(12)
+ * </pre>
+ * 
+ * Usage: <code>EnumTriangleOpt &lt;edge path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link EnumTrianglesData}.
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Custom Java objects which extend Tuple
+ * <li>Group Sorting
+ * </ul>
+ * 
+ */
+@SuppressWarnings("serial")
+public class EnumTrianglesOpt {
+
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		// set up execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// read input data
+		DataSet<Edge> edges = getEdgeDataSet(env);
+		
+		// annotate edges with degrees
+		DataSet<EdgeWithDegrees> edgesWithDegrees = edges
+				.flatMap(new EdgeDuplicator())
+				.groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new DegreeCounter())
+				.groupBy(EdgeWithDegrees.V1,EdgeWithDegrees.V2).reduce(new DegreeJoiner());
+		
+		// project edges by degrees
+		DataSet<Edge> edgesByDegree = edgesWithDegrees
+				.map(new EdgeByDegreeProjector());
+		// project edges by vertex id
+		DataSet<Edge> edgesById = edgesByDegree
+				.map(new EdgeByIdProjector());
+		
+		DataSet<Triad> triangles = edgesByDegree
+				// build triads
+				.groupBy(Edge.V1).sortGroup(Edge.V2, Order.ASCENDING).reduceGroup(new TriadBuilder())
+				// filter triads
+				.join(edgesById).where(Triad.V2,Triad.V3).equalTo(Edge.V1,Edge.V2).with(new TriadFilter());
+
+		// emit result
+		if(fileOutput) {
+			triangles.writeAsCsv(outputPath, "\n", ",");
+		} else {
+			triangles.print();
+		}
+		
+		// execute program
+		env.execute("Triangle Enumeration Example");
+		
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+	
+	/** Converts a Tuple2 into an Edge */
+	public static class TupleEdgeConverter implements MapFunction<Tuple2<Integer, Integer>, Edge> {
+		private final Edge outEdge = new Edge();
+		
+		@Override
+		public Edge map(Tuple2<Integer, Integer> t) throws Exception {
+			outEdge.copyVerticesFromTuple2(t);
+			return outEdge;
+		}
+	}
+	
+	/** Emits for an edge the original edge and its switched version. */
+	private static class EdgeDuplicator implements FlatMapFunction<Edge, Edge> {
+		
+		@Override
+		public void flatMap(Edge edge, Collector<Edge> out) throws Exception {
+			out.collect(edge);
+			edge.flipVertices();
+			out.collect(edge);
+		}
+	}
+	
+	/**
+	 * Counts the number of edges that share a common vertex.
+	 * Emits one edge for each input edge with a degree annotation for the shared vertex.
+	 * For each emitted edge, the first vertex is the vertex with the smaller id.
+	 */
+	private static class DegreeCounter implements GroupReduceFunction<Edge, EdgeWithDegrees> {
+		
+		final ArrayList<Integer> otherVertices = new ArrayList<Integer>();
+		final EdgeWithDegrees outputEdge = new EdgeWithDegrees();
+		
+		@Override
+		public void reduce(Iterable<Edge> edgesIter, Collector<EdgeWithDegrees> out) {
+			
+			Iterator<Edge> edges = edgesIter.iterator();
+			otherVertices.clear();
+			
+			// get first edge
+			Edge edge = edges.next();
+			Integer groupVertex = edge.getFirstVertex();
+			this.otherVertices.add(edge.getSecondVertex());
+			
+			// get all other edges (assumes edges are sorted by second vertex)
+			while (edges.hasNext()) {
+				edge = edges.next();
+				Integer otherVertex = edge.getSecondVertex();
+				// collect unique vertices
+				if(!otherVertices.contains(otherVertex) && otherVertex != groupVertex) {
+					this.otherVertices.add(otherVertex);
+				}
+			}
+			int degree = this.otherVertices.size();
+			
+			// emit edges
+			for(Integer otherVertex : this.otherVertices) {
+				if(groupVertex < otherVertex) {
+					outputEdge.setFirstVertex(groupVertex);
+					outputEdge.setFirstDegree(degree);
+					outputEdge.setSecondVertex(otherVertex);
+					outputEdge.setSecondDegree(0);
+				} else {
+					outputEdge.setFirstVertex(otherVertex);
+					outputEdge.setFirstDegree(0);
+					outputEdge.setSecondVertex(groupVertex);
+					outputEdge.setSecondDegree(degree);
+				}
+				out.collect(outputEdge);
+			}
+		}
+	}
+	
+	/**
+	 * Builds an edge with degree annotation from two edges that have the same vertices and only one 
+	 * degree annotation.
+	 */
+	private static class DegreeJoiner implements ReduceFunction<EdgeWithDegrees> {
+		private final EdgeWithDegrees outEdge = new EdgeWithDegrees();
+		
+		@Override
+		public EdgeWithDegrees reduce(EdgeWithDegrees edge1, EdgeWithDegrees edge2) throws Exception {
+			
+			// copy first edge
+			outEdge.copyFrom(edge1);
+			
+			// set missing degree
+			if(edge1.getFirstDegree() == 0 && edge1.getSecondDegree() != 0) {
+				outEdge.setFirstDegree(edge2.getFirstDegree());
+			} else if (edge1.getFirstDegree() != 0 && edge1.getSecondDegree() == 0) {
+				outEdge.setSecondDegree(edge2.getSecondDegree());
+			}
+			return outEdge;
+		}
+	}
+		
+	/** Projects an edge (pair of vertices) such that the first vertex is the vertex with the smaller degree. */
+	private static class EdgeByDegreeProjector implements MapFunction<EdgeWithDegrees, Edge> {
+		
+		private final Edge outEdge = new Edge();
+		
+		@Override
+		public Edge map(EdgeWithDegrees inEdge) throws Exception {
+
+			// copy vertices to simple edge
+			outEdge.copyVerticesFromEdgeWithDegrees(inEdge);
+
+			// flip vertices if first degree is larger than second degree.
+			if(inEdge.getFirstDegree() > inEdge.getSecondDegree()) {
+				outEdge.flipVertices();
+			}
+
+			// return edge
+			return outEdge;
+		}
+	}
+	
+	/** Projects an edge (pair of vertices) such that the id of the first is smaller than the id of the second. */
+	private static class EdgeByIdProjector implements MapFunction<Edge, Edge> {
+	
+		@Override
+		public Edge map(Edge inEdge) throws Exception {
+			
+			// flip vertices if necessary
+			if(inEdge.getFirstVertex() > inEdge.getSecondVertex()) {
+				inEdge.flipVertices();
+			}
+			
+			return inEdge;
+		}
+	}
+	
+	/**
+	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
+	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
+	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
+	 */
+	private static class TriadBuilder implements GroupReduceFunction<Edge, Triad> {
+		
+		private final List<Integer> vertices = new ArrayList<Integer>();
+		private final Triad outTriad = new Triad();
+		
+		@Override
+		public void reduce(Iterable<Edge> edgesIter, Collector<Triad> out) throws Exception {
+			final Iterator<Edge> edges = edgesIter.iterator();
+			
+			// clear vertex list
+			vertices.clear();
+			
+			// read first edge
+			Edge firstEdge = edges.next();
+			outTriad.setFirstVertex(firstEdge.getFirstVertex());
+			vertices.add(firstEdge.getSecondVertex());
+			
+			// build and emit triads
+			while (edges.hasNext()) {
+				Integer higherVertexId = edges.next().getSecondVertex();
+				
+				// combine vertex with all previously read vertices
+				for(Integer lowerVertexId : vertices) {
+					outTriad.setSecondVertex(lowerVertexId);
+					outTriad.setThirdVertex(higherVertexId);
+					out.collect(outTriad);
+				}
+				vertices.add(higherVertexId);
+			}
+		}
+	}
+	
+	/** Filters triads (three vertices connected by two edges) without a closing third edge. */
+	private static class TriadFilter implements JoinFunction<Triad, Edge, Triad> {
+		
+		@Override
+		public Triad join(Triad triad, Edge edge) throws Exception {
+			return triad;
+		}
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean fileOutput = false;
+	private static String edgePath = null;
+	private static String outputPath = null;
+	
+	private static boolean parseParameters(String[] args) {
+		
+		if(args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(args.length == 2) {
+				edgePath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing Enum Triangles Opt example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: EnumTriangleOpt <edge path> <result path>");
+		}
+		return true;
+	}
+	
+	private static DataSet<Edge> getEdgeDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return env.readCsvFile(edgePath)
+						.fieldDelimiter(' ')
+						.includeFields(true, true)
+						.types(Integer.class, Integer.class)
+						.map(new TupleEdgeConverter());
+		} else {
+			return EnumTrianglesData.getDefaultEdgeDataSet(env);
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
new file mode 100644
index 0000000..4a491b4
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
@@ -0,0 +1,288 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.graph;
+
+import static org.apache.flink.api.java.aggregation.Aggregations.SUM;
+
+import java.util.ArrayList;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.examples.java.graph.util.PageRankData;
+
+/**
+ * A basic implementation of the Page Rank algorithm using a bulk iteration.
+ * 
+ * <p>
+ * This implementation requires a set of pages and a set of directed links as input and works as follows. <br> 
+ * In each iteration, the rank of every page is evenly distributed to all pages it points to.
+ * Each page collects the partial ranks of all pages that point to it, sums them up, and applies a dampening factor to the sum.
+ * The result is the new rank of the page. A new iteration is started with the new ranks of all pages.
+ * This implementation terminates after a fixed number of iterations.<br>
+ * This is the Wikipedia entry for the <a href="http://en.wikipedia.org/wiki/Page_rank">Page Rank algorithm</a>. 
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Pages represented as an (long) ID separated by new-line characters.<br> 
+ * For example <code>"1\n2\n12\n42\n63\n"</code> gives five pages with IDs 1, 2, 12, 42, and 63.
+ * <li>Links are represented as pairs of page IDs which are separated by space 
+ * characters. Links are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).<br>
+ * For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+ * </ul>
+ * 
+ * <p>
+ * Usage: <code>PageRankBasic &lt;pages path&gt; &lt;links path&gt; &lt;output path&gt; &lt;num pages&gt; &lt;num iterations&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link PageRankData} and 10 iterations.
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Bulk Iterations
+ * <li>Default Join
+ * <li>Configure user-defined functions using constructor parameters.
+ * </ul> 
+ * 
+ *
+ */
+@SuppressWarnings("serial")
+public class PageRankBasic {
+	
+	private static final double DAMPENING_FACTOR = 0.85;
+	private static final double EPSILON = 0.0001;
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		// set up execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// get input data
+		DataSet<Long> pagesInput = getPagesDataSet(env);
+		DataSet<Tuple2<Long, Long>> linksInput = getLinksDataSet(env);
+		
+		// assign initial rank to pages
+		DataSet<Tuple2<Long, Double>> pagesWithRanks = pagesInput.
+				map(new RankAssigner((1.0d / numPages)));
+		
+		// build adjacency list from link input
+		DataSet<Tuple2<Long, Long[]>> adjacencyListInput = 
+				linksInput.groupBy(0).reduceGroup(new BuildOutgoingEdgeList());
+		
+		// set iterative data set
+		IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(maxIterations);
+		
+		DataSet<Tuple2<Long, Double>> newRanks = iteration
+				// join pages with outgoing edges and distribute rank
+				.join(adjacencyListInput).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch())
+				// collect and sum ranks
+				.groupBy(0).aggregate(SUM, 1)
+				// apply dampening factor
+				.map(new Dampener(DAMPENING_FACTOR, numPages));
+		
+		DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
+				newRanks, 
+				newRanks.join(iteration).where(0).equalTo(0)
+				// termination condition
+				.filter(new EpsilonFilter()));
+
+		// emit result
+		if(fileOutput) {
+			finalPageRanks.writeAsCsv(outputPath, "\n", " ");
+		} else {
+			finalPageRanks.print();
+		}
+
+		// execute program
+		env.execute("Basic Page Rank Example");
+		
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/** 
+	 * A map function that assigns an initial rank to all pages. 
+	 */
+	public static final class RankAssigner implements MapFunction<Long, Tuple2<Long, Double>> {
+		Tuple2<Long, Double> outPageWithRank;
+		
+		public RankAssigner(double rank) {
+			this.outPageWithRank = new Tuple2<Long, Double>(-1l, rank);
+		}
+		
+		@Override
+		public Tuple2<Long, Double> map(Long page) {
+			outPageWithRank.f0 = page;
+			return outPageWithRank;
+		}
+	}
+	
+	/**
+	 * A reduce function that takes a sequence of edges and builds the adjacency list for the vertex where the edges
+	 * originate. Run as a pre-processing step.
+	 */
+	@ConstantFields("0")
+	public static final class BuildOutgoingEdgeList implements GroupReduceFunction<Tuple2<Long, Long>, Tuple2<Long, Long[]>> {
+		
+		private final ArrayList<Long> neighbors = new ArrayList<Long>();
+		
+		@Override
+		public void reduce(Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long[]>> out) {
+			neighbors.clear();
+			Long id = 0L;
+			
+			for (Tuple2<Long, Long> n : values) {
+				id = n.f0;
+				neighbors.add(n.f1);
+			}
+			out.collect(new Tuple2<Long, Long[]>(id, neighbors.toArray(new Long[neighbors.size()])));
+		}
+	}
+	
+	/**
+	 * Join function that distributes a fraction of a vertex's rank to all neighbors.
+	 */
+	public static final class JoinVertexWithEdgesMatch implements FlatMapFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>>, Tuple2<Long, Double>> {
+
+		@Override
+		public void flatMap(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>> value, Collector<Tuple2<Long, Double>> out){
+			Long[] neigbors = value.f1.f1;
+			double rank = value.f0.f1;
+			double rankToDistribute = rank / ((double) neigbors.length);
+				
+			for (int i = 0; i < neigbors.length; i++) {
+				out.collect(new Tuple2<Long, Double>(neigbors[i], rankToDistribute));
+			}
+		}
+	}
+	
+	/**
+	 * The function that applies the page rank dampening formula
+	 */
+	@ConstantFields("0")
+	public static final class Dampener implements MapFunction<Tuple2<Long,Double>, Tuple2<Long,Double>> {
+
+		private final double dampening;
+		private final double randomJump;
+		
+		public Dampener(double dampening, double numVertices) {
+			this.dampening = dampening;
+			this.randomJump = (1 - dampening) / numVertices;
+		}
+
+		@Override
+		public Tuple2<Long, Double> map(Tuple2<Long, Double> value) {
+			value.f1 = (value.f1 * dampening) + randomJump;
+			return value;
+		}
+	}
+	
+	/**
+	 * Filter that filters vertices where the rank difference is below a threshold.
+	 */
+	public static final class EpsilonFilter implements FilterFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>>> {
+
+		@Override
+		public boolean filter(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>> value) {
+			return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
+		}
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean fileOutput = false;
+	private static String pagesInputPath = null;
+	private static String linksInputPath = null;
+	private static String outputPath = null;
+	private static long numPages = 0;
+	private static int maxIterations = 10;
+	
+	private static boolean parseParameters(String[] args) {
+		
+		if(args.length > 0) {
+			if(args.length == 5) {
+				fileOutput = true;
+				pagesInputPath = args[0];
+				linksInputPath = args[1];
+				outputPath = args[2];
+				numPages = Integer.parseInt(args[3]);
+				maxIterations = Integer.parseInt(args[4]);
+			} else {
+				System.err.println("Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing PageRank Basic example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
+			
+			numPages = PageRankData.getNumberOfPages();
+		}
+		return true;
+	}
+	
+	private static DataSet<Long> getPagesDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return env
+						.readCsvFile(pagesInputPath)
+							.fieldDelimiter(' ')
+							.lineDelimiter("\n")
+							.types(Long.class)
+						.map(new MapFunction<Tuple1<Long>, Long>() {
+							@Override
+							public Long map(Tuple1<Long> v) { return v.f0; }
+						});
+		} else {
+			return PageRankData.getDefaultPagesDataSet(env);
+		}
+	}
+	
+	private static DataSet<Tuple2<Long, Long>> getLinksDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return env.readCsvFile(linksInputPath)
+						.fieldDelimiter(' ')
+						.lineDelimiter("\n")
+						.types(Long.class, Long.class);
+		} else {
+			return PageRankData.getDefaultEdgeDataSet(env);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
new file mode 100644
index 0000000..30230d6
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.graph;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.api.common.ProgramDescription;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.examples.java.graph.util.ConnectedComponentsData;
+import org.apache.flink.util.Collector;
+
+import java.util.HashSet;
+import java.util.Set;
+
+@SuppressWarnings("serial")
+public class TransitiveClosureNaive implements ProgramDescription {
+
+
+	public static void main (String... args) throws Exception{
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up execution environment
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env);
+
+		IterativeDataSet<Tuple2<Long,Long>> paths = edges.iterate(maxIterations);
+
+		DataSet<Tuple2<Long,Long>> nextPaths = paths
+				.join(edges)
+				.where(1)
+				.equalTo(0)
+				.with(new JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>>() {
+					@Override
+					/**
+						left: Path (z,x) - x is reachable by z
+						right: Edge (x,y) - edge x-->y exists
+						out: Path (z,y) - y is reachable by z
+					 */
+					public Tuple2<Long, Long> join(Tuple2<Long, Long> left, Tuple2<Long, Long> right) throws Exception {
+						return new Tuple2<Long, Long>(
+								new Long(left.f0),
+								new Long(right.f1));
+					}
+				})
+				.union(paths)
+				.groupBy(0, 1)
+				.reduceGroup(new GroupReduceFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>() {
+					@Override
+					public void reduce(Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) throws Exception {
+						out.collect(values.iterator().next());
+					}
+				});
+
+		DataSet<Tuple2<Long,Long>> newPaths = paths
+				.coGroup(nextPaths)
+				.where(0).equalTo(0)
+				.with(new CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>>() {
+					Set prevSet = new HashSet<Tuple2<Long,Long>>();
+					@Override
+					public void coGroup(Iterable<Tuple2<Long, Long>> prevPaths, Iterable<Tuple2<Long, Long>> nextPaths, Collector<Tuple2<Long, Long>> out) throws Exception {
+						for (Tuple2<Long,Long> prev : prevPaths) {
+							prevSet.add(prev);
+						}
+						for (Tuple2<Long,Long> next: nextPaths) {
+							if (!prevSet.contains(next)) {
+								out.collect(next);
+							}
+						}
+					}
+				});
+
+		DataSet<Tuple2<Long, Long>> transitiveClosure = paths.closeWith(nextPaths);
+
+
+		// emit result
+		if (fileOutput) {
+			transitiveClosure.writeAsCsv(outputPath, "\n", " ");
+		} else {
+			transitiveClosure.print();
+		}
+
+		// execute program
+		env.execute("Transitive Closure Example");
+
+	}
+
+	@Override
+	public String getDescription() {
+		return "Parameters: <edges-path> <result-path> <max-number-of-iterations>";
+	}
+
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String edgesPath = null;
+	private static String outputPath = null;
+	private static int maxIterations = 10;
+
+	private static boolean parseParameters(String[] programArguments) {
+
+		if (programArguments.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (programArguments.length == 3) {
+				edgesPath = programArguments[0];
+				outputPath = programArguments[1];
+				maxIterations = Integer.parseInt(programArguments[2]);
+			} else {
+				System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of iterations>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing TransitiveClosure example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: TransitiveClosure <edges path> <result path> <max number of iterations>");
+		}
+		return true;
+	}
+
+
+	private static DataSet<Tuple2<Long, Long>> getEdgeDataSet(ExecutionEnvironment env) {
+
+		if(fileOutput) {
+			return env.readCsvFile(edgesPath).fieldDelimiter(' ').types(Long.class, Long.class);
+		} else {
+			return ConnectedComponentsData.getDefaultEdgeDataSet(env);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
new file mode 100644
index 0000000..27c7d45
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.graph.util;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * Provides the default data sets used for the Connected Components example program.
+ * The default data sets are used, if no parameters are given to the program.
+ *
+ */
+public class ConnectedComponentsData {
+	
+	public static final Object[][] VERTICES  = new Object[][] {
+		new Object[]{1L}, new Object[]{2L}, new Object[]{3L}, new Object[]{4L}, 
+		new Object[]{5L},new Object[]{6L}, new Object[]{7L}, new Object[]{8L}, 
+		new Object[]{9L}, new Object[]{10L}, new Object[]{11L}, new Object[]{12L}, 
+		new Object[]{13L}, new Object[]{14L}, new Object[]{15L}, new Object[]{16L}
+	};
+
+	public static DataSet<Long> getDefaultVertexDataSet(ExecutionEnvironment env) {
+		List<Long> verticesList = new LinkedList<Long>();
+		for (Object[] vertex : VERTICES) {
+			verticesList.add((Long) vertex[0]);
+		}
+		return env.fromCollection(verticesList);
+	}
+	
+	public static final Object[][] EDGES = new Object[][] {
+		new Object[]{1L, 2L},
+		new Object[]{2L, 3L},
+		new Object[]{2L, 4L},
+		new Object[]{3L, 5L},
+		new Object[]{6L, 7L},
+		new Object[]{8L, 9L},
+		new Object[]{8L, 10L},
+		new Object[]{5L, 11L},
+		new Object[]{11L, 12L},
+		new Object[]{10L, 13L},
+		new Object[]{9L, 14L},
+		new Object[]{13L, 14L},
+		new Object[]{1L, 15L},
+		new Object[]{16L, 1L}
+	};
+	
+	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+		
+		List<Tuple2<Long, Long>> edgeList = new LinkedList<Tuple2<Long, Long>>();
+		for (Object[] edge : EDGES) {
+			edgeList.add(new Tuple2<Long, Long>((Long) edge[0], (Long) edge[1]));
+		}
+		return env.fromCollection(edgeList);
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java
new file mode 100644
index 0000000..257b7d8
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.graph.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Edge;
+
+/**
+ * Provides the default data sets used for the Triangle Enumeration example programs.
+ * The default data sets are used, if no parameters are given to the program.
+ *
+ */
+public class EnumTrianglesData {
+
+	public static final Object[][] EDGES = {
+		{1, 2},
+		{1, 3},
+		{1 ,4},
+		{1, 5},
+		{2, 3},
+		{2, 5},
+		{3, 4},
+		{3, 7},
+		{3, 8},
+		{5, 6},
+		{7, 8}
+	};
+	
+	public static DataSet<Edge> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+		
+		List<Edge> edges = new ArrayList<Edge>();
+		for(Object[] e : EDGES) {
+			edges.add(new Edge((Integer)e[0], (Integer)e[1]));
+		}
+		
+		return env.fromCollection(edges);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java
new file mode 100644
index 0000000..d7eefae
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.graph.util;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+
+public class EnumTrianglesDataTypes {
+
+	public static class Edge extends Tuple2<Integer, Integer> {
+		private static final long serialVersionUID = 1L;
+		
+		public static final int V1 = 0;
+		public static final int V2 = 1;
+		
+		public Edge() {}
+		
+		public Edge(final Integer v1, final Integer v2) {
+			this.setFirstVertex(v1);
+			this.setSecondVertex(v2);
+		}
+		
+		public Integer getFirstVertex() { return this.getField(V1); }
+		
+		public Integer getSecondVertex() { return this.getField(V2); }
+		
+		public void setFirstVertex(final Integer vertex1) { this.setField(vertex1, V1); }
+		
+		public void setSecondVertex(final Integer vertex2) { this.setField(vertex2, V2); }
+		
+		public void copyVerticesFromTuple2(Tuple2<Integer, Integer> t) {
+			this.setFirstVertex(t.f0);
+			this.setSecondVertex(t.f1);
+		}
+		
+		public void copyVerticesFromEdgeWithDegrees(EdgeWithDegrees ewd) {
+			this.setFirstVertex(ewd.getFirstVertex());
+			this.setSecondVertex(ewd.getSecondVertex());
+		}
+		
+		public void flipVertices() {
+			Integer tmp = this.getFirstVertex();
+			this.setFirstVertex(this.getSecondVertex());
+			this.setSecondVertex(tmp);
+		}
+	}
+	
+	public static class Triad extends Tuple3<Integer, Integer, Integer> {
+		private static final long serialVersionUID = 1L;
+		
+		public static final int V1 = 0;
+		public static final int V2 = 1;
+		public static final int V3 = 2;
+		
+		public Triad() {}
+		
+		public void setFirstVertex(final Integer vertex1) { this.setField(vertex1, V1); }
+		
+		public void setSecondVertex(final Integer vertex2) { this.setField(vertex2, V2); }
+		
+		public void setThirdVertex(final Integer vertex3) { this.setField(vertex3, V3); }
+	}
+	
+	public static class EdgeWithDegrees extends Tuple4<Integer, Integer, Integer, Integer> {
+		private static final long serialVersionUID = 1L;
+
+		public static final int V1 = 0;
+		public static final int V2 = 1;
+		public static final int D1 = 2;
+		public static final int D2 = 3;
+		
+		public EdgeWithDegrees() { }
+			
+		public Integer getFirstVertex() { return this.getField(V1); }
+		
+		public Integer getSecondVertex() { return this.getField(V2); }
+		
+		public Integer getFirstDegree() { return this.getField(D1); }
+		
+		public Integer getSecondDegree() { return this.getField(D2); }
+		
+		public void setFirstVertex(final Integer vertex1) { this.setField(vertex1, V1); }
+		
+		public void setSecondVertex(final Integer vertex2) { this.setField(vertex2, V2); }
+		
+		public void setFirstDegree(final Integer degree1) { this.setField(degree1, D1); }
+		
+		public void setSecondDegree(final Integer degree2) { this.setField(degree2, D2); }
+		
+		public void copyFrom(final EdgeWithDegrees edge) {
+			this.setFirstVertex(edge.getFirstVertex());
+			this.setSecondVertex(edge.getSecondVertex());
+			this.setFirstDegree(edge.getFirstDegree());
+			this.setSecondDegree(edge.getSecondDegree());
+		}
+	}
+	
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java
new file mode 100644
index 0000000..d6ee274
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.graph.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/**
+ * Provides the default data sets used for the PageRank example program.
+ * The default data sets are used, if no parameters are given to the program.
+ *
+ */
+public class PageRankData {
+
+	public static final Object[][] EDGES = {
+		{1L, 2L},
+		{1L, 15L},
+		{2L, 3L},
+		{2L, 4L},
+		{2L, 5L},
+		{2L, 6L},
+		{2L, 7L},
+		{3L, 13L},
+		{4L, 2L},
+		{5L, 11L},
+		{5L, 12L},
+		{6L, 1L},
+		{6L, 7L},
+		{6L, 8L},
+		{7L, 1L},
+		{7L, 8L},
+		{8L, 1L},
+		{8L, 9L},
+		{8L, 10L},
+		{9L, 14L},
+		{9L, 1L},
+		{10L, 1L},
+		{10L, 13L},
+		{11L, 12L},
+		{11L, 1L},
+		{12L, 1L},
+		{13L, 14L},
+		{14L, 12L},
+		{15L, 1L},
+	};
+	
+	private static long numPages = 15;
+	
+	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+		
+		List<Tuple2<Long, Long>> edges = new ArrayList<Tuple2<Long, Long>>();
+		for(Object[] e : EDGES) {
+			edges.add(new Tuple2<Long, Long>((Long)e[0], (Long)e[1]));
+		}
+		return env.fromCollection(edges);
+	}
+	
+	public static DataSet<Long> getDefaultPagesDataSet(ExecutionEnvironment env) {
+		return env.generateSequence(1, 15);
+	}
+	
+	public static long getNumberOfPages() {
+		return numPages;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java
new file mode 100644
index 0000000..0d4fd3e
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.misc;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/** 
+ * Estimates the value of Pi using the Monte Carlo method.
+ * The area of a circle is Pi * R^2, R being the radius of the circle 
+ * The area of a square is 4 * R^2, where the length of the square's edge is 2*R.
+ * 
+ * Thus Pi = 4 * (area of circle / area of square).
+ * 
+ * The idea is to find a way to estimate the circle to square area ratio.
+ * The Monte Carlo method suggests collecting random points (within the square)
+ * and then counting the number of points that fall within the circle
+ * 
+ * <pre>
+ * {@code
+ * x = Math.random()
+ * y = Math.random()
+ * 
+ * x * x + y * y < 1
+ * }
+ * </pre>
+ */
+@SuppressWarnings("serial")
+public class PiEstimation implements java.io.Serializable {
+	
+	
+	public static void main(String[] args) throws Exception {
+
+		final long numSamples = args.length > 0 ? Long.parseLong(args[0]) : 1000000;
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// count how many of the samples would randomly fall into
+		// the unit circle
+		DataSet<Long> count = 
+				env.generateSequence(1, numSamples)
+				.map(new Sampler())
+				.reduce(new SumReducer());
+
+		// the ratio of the unit circle surface to 4 times the unit square is pi
+		DataSet<Double> pi = count
+				.map(new MapFunction<Long, Double>() {
+					public Double map(Long value) {
+						return value * 4.0 / numSamples;
+					}
+				});
+
+		System.out.println("We estimate Pi to be:");
+		pi.print();
+
+		env.execute();
+	}
+
+	//*************************************************************************
+	//     USER FUNCTIONS
+	//*************************************************************************
+	
+	
+	/** 
+	 * Sampler randomly emits points that fall within a square of edge x * y.
+	 * It calculates the distance to the center of a virtually centered circle of radius x = y = 1
+	 * If the distance is less than 1, then and only then does it returns a 1.
+	 */
+	public static class Sampler implements MapFunction<Long, Long> {
+
+		@Override
+		public Long map(Long value) throws Exception{
+			double x = Math.random();
+			double y = Math.random();
+			return (x * x + y * y) < 1 ? 1L : 0L;
+		}
+	}
+
+	
+	/** 
+	 * Simply sums up all long values.
+	 */
+	public static final class SumReducer implements ReduceFunction<Long>{
+
+		@Override
+		public Long reduce(Long value1, Long value2) throws Exception {
+			return value1 + value2;
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
new file mode 100644
index 0000000..7940310
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
@@ -0,0 +1,316 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.ml;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.examples.java.ml.util.LinearRegressionData;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+
+/**
+ * This example implements a basic Linear Regression  to solve the y = theta0 + theta1*x problem using batch gradient descent algorithm.
+ *
+ * <p>
+ * Linear Regression with BGD(batch gradient descent) algorithm is an iterative clustering algorithm and works as follows:<br>
+ * Giving a data set and target set, the BGD try to find out the best parameters for the data set to fit the target set.
+ * In each iteration, the algorithm computes the gradient of the cost function and use it to update all the parameters.
+ * The algorithm terminates after a fixed number of iterations (as in this implementation)
+ * With enough iteration, the algorithm can minimize the cost function and find the best parameters
+ * This is the Wikipedia entry for the <a href = "http://en.wikipedia.org/wiki/Linear_regression">Linear regression</a> and <a href = "http://en.wikipedia.org/wiki/Gradient_descent">Gradient descent algorithm</a>.
+ * 
+ * <p>
+ * This implementation works on one-dimensional data. And find the two-dimensional theta.<br>
+ * It find the best Theta parameter to fit the target.
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Data points are represented as two double values separated by a blank character. The first one represent the X(the training data) and the second represent the Y(target).
+ * Data points are separated by newline characters.<br>
+ * For example <code>"-0.02 -0.04\n5.3 10.6\n"</code> gives two data points (x=-0.02, y=-0.04) and (x=5.3, y=10.6).
+ * </ul>
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li> Bulk iterations
+ * <li> Broadcast variables in bulk iterations
+ * <li> Custom Java objects (PoJos)
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class LinearRegression {
+
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception{
+
+		if(!parseParameters(args)) {
+			return;
+		}
+
+		// set up execution environment
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// get input x data from elements
+		DataSet<Data> data = getDataSet(env);
+
+		// get the parameters from elements
+		DataSet<Params> parameters = getParamsDataSet(env);
+
+		// set number of bulk iterations for SGD linear Regression
+		IterativeDataSet<Params> loop = parameters.iterate(numIterations);
+
+		DataSet<Params> new_parameters = data
+				// compute a single step using every sample
+				.map(new SubUpdate()).withBroadcastSet(loop, "parameters")
+				// sum up all the steps
+				.reduce(new UpdateAccumulator())
+				// average the steps and update all parameters
+				.map(new Update());
+
+		// feed new parameters back into next iteration
+		DataSet<Params> result = loop.closeWith(new_parameters);
+
+		// emit result
+		if(fileOutput) {
+			result.writeAsText(outputPath);
+		} else {
+			result.print();
+		}
+
+		// execute program
+		env.execute("Linear Regression example");
+
+	}
+
+	// *************************************************************************
+	//     DATA TYPES
+	// *************************************************************************
+
+	/**
+	 * A simple data sample, x means the input, and y means the target.
+	 */
+	public static class Data implements Serializable{
+		public double x,y;
+
+		public Data() {};
+
+		public Data(double x ,double y){
+			this.x = x;
+			this.y = y;
+		}
+
+		@Override
+		public String toString() {
+			return "(" + x + "|" + y + ")";
+		}
+
+	}
+
+	/**
+	 * A set of parameters -- theta0, theta1.
+	 */
+	public static class Params implements Serializable{
+
+		private double theta0,theta1;
+
+		public Params(){};
+
+		public Params(double x0, double x1){
+			this.theta0 = x0;
+			this.theta1 = x1;
+		}
+
+		@Override
+		public String toString() {
+			return theta0 + " " + theta1;
+		}
+
+		public double getTheta0() {
+			return theta0;
+		}
+
+		public double getTheta1() {
+			return theta1;
+		}
+
+		public void setTheta0(double theta0) {
+			this.theta0 = theta0;
+		}
+
+		public void setTheta1(double theta1) {
+			this.theta1 = theta1;
+		}
+
+		public Params div(Integer a){
+			this.theta0 = theta0 / a ;
+			this.theta1 = theta1 / a ;
+			return this;
+		}
+
+	}
+
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/** Converts a Tuple2<Double,Double> into a Data. */
+	public static final class TupleDataConverter implements MapFunction<Tuple2<Double, Double>, Data> {
+
+		@Override
+		public Data map(Tuple2<Double, Double> t) throws Exception {
+			return new Data(t.f0, t.f1);
+		}
+	}
+
+	/** Converts a Tuple2<Double,Double> into a Params. */
+	public static final class TupleParamsConverter implements MapFunction<Tuple2<Double, Double>,Params> {
+
+		@Override
+		public Params map(Tuple2<Double, Double> t)throws Exception {
+			return new Params(t.f0,t.f1);
+		}
+	}
+
+	/**
+	 * Compute a single BGD type update for every parameters.
+	 */
+	public static class SubUpdate extends RichMapFunction<Data,Tuple2<Params,Integer>> {
+
+		private Collection<Params> parameters; 
+
+		private Params parameter;
+
+		private int count = 1;
+
+		/** Reads the parameters from a broadcast variable into a collection. */
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			this.parameters = getRuntimeContext().getBroadcastVariable("parameters");
+		}
+
+		@Override
+		public Tuple2<Params, Integer> map(Data in) throws Exception {
+
+			for(Params p : parameters){
+				this.parameter = p; 
+			}
+
+			double theta_0 = parameter.theta0 - 0.01*((parameter.theta0 + (parameter.theta1*in.x)) - in.y);
+			double theta_1 = parameter.theta1 - 0.01*(((parameter.theta0 + (parameter.theta1*in.x)) - in.y) * in.x);
+
+			return new Tuple2<Params,Integer>(new Params(theta_0,theta_1),count);
+		}
+	}
+
+	/**  
+	 * Accumulator all the update.
+	 * */
+	public static class UpdateAccumulator implements ReduceFunction<Tuple2<Params, Integer>> {
+
+		@Override
+		public Tuple2<Params, Integer> reduce(Tuple2<Params, Integer> val1, Tuple2<Params, Integer> val2) {
+
+			double new_theta0 = val1.f0.theta0 + val2.f0.theta0;
+			double new_theta1 = val1.f0.theta1 + val2.f0.theta1;
+			Params result = new Params(new_theta0,new_theta1);
+			return new Tuple2<Params, Integer>( result, val1.f1 + val2.f1);
+
+		}
+	}
+
+	/**
+	 * Compute the final update by average them.
+	 */
+	public static class Update implements MapFunction<Tuple2<Params, Integer>,Params> {
+
+		@Override
+		public Params map(Tuple2<Params, Integer> arg0) throws Exception {
+
+			return arg0.f0.div(arg0.f1);
+
+		}
+
+	}
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String dataPath = null;
+	private static String outputPath = null;
+	private static int numIterations = 10;
+
+	private static boolean parseParameters(String[] programArguments) {
+
+		if(programArguments.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(programArguments.length == 3) {
+				dataPath = programArguments[0];
+				outputPath = programArguments[1];
+				numIterations = Integer.parseInt(programArguments[2]);
+			} else {
+				System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing Linear Regression example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  We provide a data generator to create synthetic input files for this program.");
+			System.out.println("  Usage: LinearRegression <data path> <result path> <num iterations>");
+		}
+		return true;
+	}
+
+	private static DataSet<Data> getDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			// read data from CSV file
+			return env.readCsvFile(dataPath)
+					.fieldDelimiter(' ')
+					.includeFields(true, true)
+					.types(Double.class, Double.class)
+					.map(new TupleDataConverter());
+		} else {
+			return LinearRegressionData.getDefaultDataDataSet(env);
+		}
+	}
+
+	private static DataSet<Params> getParamsDataSet(ExecutionEnvironment env) {
+
+		return LinearRegressionData.getDefaultParamsDataSet(env);
+
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionData.java
new file mode 100644
index 0000000..84d332f
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionData.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.ml.util;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.ml.LinearRegression.Data;
+import org.apache.flink.examples.java.ml.LinearRegression.Params;
+
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the Linear Regression example
+ * program. The default data sets are used, if no parameters are given to the
+ * program.
+ */
+public class LinearRegressionData {
+
+	// We have the data as object arrays so that we can also generate Scala Data
+	// Sources from it.
+	public static final Object[][] PARAMS = new Object[][] { new Object[] {
+			0.0, 0.0 } };
+
+	public static final Object[][] DATA = new Object[][] {
+			new Object[] { 0.5, 1.0 }, new Object[] { 1.0, 2.0 },
+			new Object[] { 2.0, 4.0 }, new Object[] { 3.0, 6.0 },
+			new Object[] { 4.0, 8.0 }, new Object[] { 5.0, 10.0 },
+			new Object[] { 6.0, 12.0 }, new Object[] { 7.0, 14.0 },
+			new Object[] { 8.0, 16.0 }, new Object[] { 9.0, 18.0 },
+			new Object[] { 10.0, 20.0 }, new Object[] { -0.08, -0.16 },
+			new Object[] { 0.13, 0.26 }, new Object[] { -1.17, -2.35 },
+			new Object[] { 1.72, 3.45 }, new Object[] { 1.70, 3.41 },
+			new Object[] { 1.20, 2.41 }, new Object[] { -0.59, -1.18 },
+			new Object[] { 0.28, 0.57 }, new Object[] { 1.65, 3.30 },
+			new Object[] { -0.55, -1.08 } };
+
+	public static DataSet<Params> getDefaultParamsDataSet(
+			ExecutionEnvironment env) {
+		List<Params> paramsList = new LinkedList<Params>();
+		for (Object[] params : PARAMS) {
+			paramsList.add(new Params((Double) params[0], (Double) params[1]));
+		}
+		return env.fromCollection(paramsList);
+	}
+
+	public static DataSet<Data> getDefaultDataDataSet(ExecutionEnvironment env) {
+
+		List<Data> dataList = new LinkedList<Data>();
+		for (Object[] data : DATA) {
+			dataList.add(new Data((Double) data[0], (Double) data[1]));
+		}
+		return env.fromCollection(dataList);
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionDataGenerator.java
new file mode 100644
index 0000000..148c607
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/util/LinearRegressionDataGenerator.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.ml.util;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.Locale;
+import java.util.Random;
+
+/**
+ * Generates data for the {@link org.apache.flink.examples.java.ml.LinearRegression} example program.
+ */
+public class LinearRegressionDataGenerator {
+
+	static {
+		Locale.setDefault(Locale.US);
+	}
+
+	private static final String POINTS_FILE = "data";
+	private static final long DEFAULT_SEED = 4650285087650871364L;
+	private static final int DIMENSIONALITY = 1;
+	private static final DecimalFormat FORMAT = new DecimalFormat("#0.00");
+	private static final char DELIMITER = ' ';
+
+	/**
+	 * Main method to generate data for the {@link org.apache.flink.examples.java.ml.LinearRegression} example program.
+	 * <p>
+	 * The generator creates to files:
+	 * <ul>
+	 * <li><code>{tmp.dir}/data</code> for the data points
+	 * </ul> 
+	 * 
+	 * @param args 
+	 * <ol>
+	 * <li>Int: Number of data points
+	 * <li><b>Optional</b> Long: Random seed
+	 * </ol>
+	 */
+	public static void main(String[] args) throws IOException {
+
+		// check parameter count
+		if (args.length < 1) {
+			System.out.println("LinearRegressionDataGenerator <numberOfDataPoints> [<seed>]");
+			System.exit(1);
+		}
+
+		// parse parameters
+		final int numDataPoints = Integer.parseInt(args[0]);
+		final long firstSeed = args.length > 1 ? Long.parseLong(args[4]) : DEFAULT_SEED;
+		final Random random = new Random(firstSeed);
+		final String tmpDir = System.getProperty("java.io.tmpdir");
+
+		// write the points out
+		BufferedWriter pointsOut = null;
+		try {
+			pointsOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+POINTS_FILE)));
+			StringBuilder buffer = new StringBuilder();
+
+			// DIMENSIONALITY + 1 means that the number of x(dimensionality) and target y
+			double[] point = new double[DIMENSIONALITY+1];
+
+			for (int i = 1; i <= numDataPoints; i++) {
+				point[0] = random.nextGaussian();
+				point[1] = 2 * point[0] + 0.01*random.nextGaussian();
+				writePoint(point, buffer, pointsOut);
+			}
+
+		}
+		finally {
+			if (pointsOut != null) {
+				pointsOut.close();
+			}
+		}
+
+		System.out.println("Wrote "+numDataPoints+" data points to "+tmpDir+"/"+POINTS_FILE);
+	}
+
+
+	private static void writePoint(double[] data, StringBuilder buffer, BufferedWriter out) throws IOException {
+		buffer.setLength(0);
+
+		// write coordinates
+		for (int j = 0; j < data.length; j++) {
+			buffer.append(FORMAT.format(data[j]));
+			if(j < data.length - 1) {
+				buffer.append(DELIMITER);
+			}
+		}
+
+		out.write(buffer.toString());
+		out.newLine();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java
new file mode 100644
index 0000000..6faafe0
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java
@@ -0,0 +1,261 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.java.relational;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+
+/**
+ * This program filters lines from a CSV file with empty fields. In doing so, it counts the number of empty fields per
+ * column within a CSV file using a custom accumulator for vectors. In this context, empty fields are those, that at
+ * most contain whitespace characters like space and tab.
+ * <p>
+ * The input file is a plain text CSV file with the semicolon as field separator and double quotes as field delimiters
+ * and three columns. See {@link #getDataSet(ExecutionEnvironment)} for configuration.
+ * <p>
+ * Usage: <code>FilterAndCountIncompleteLines [&lt;input file path&gt; [&lt;result path&gt;]]</code> <br>
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>custom accumulators
+ * <li>tuple data types
+ * <li>inline-defined functions
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class EmptyFieldsCountAccumulator {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	private static final String EMPTY_FIELD_ACCUMULATOR = "empty-fields";
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// get the data set
+		final DataSet<Tuple> file = getDataSet(env);
+
+		// filter lines with empty fields
+		final DataSet<Tuple> filteredLines = file.filter(new EmptyFieldFilter());
+
+		// Here, we could do further processing with the filtered lines...
+		
+		// output the filtered lines
+		if (outputPath == null) {
+			filteredLines.print();
+		} else {
+			filteredLines.writeAsCsv(outputPath);
+		}
+
+		// execute program
+		final JobExecutionResult result = env.execute("Accumulator example");
+
+		// get the accumulator result via its registration key
+		final List<Integer> emptyFields = result.getAccumulatorResult(EMPTY_FIELD_ACCUMULATOR);
+		System.out.format("Number of detected empty fields per column: %s\n", emptyFields);
+
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static String filePath;
+	private static String outputPath;
+
+	private static boolean parseParameters(final String[] programArguments) {
+
+		if (programArguments.length >= 3) {
+			System.err.println("Usage: FilterAndCountIncompleteLines [<input file path> [<result path>]]");
+			return false;
+		}
+
+		if (programArguments.length >= 1) {
+			filePath = programArguments[0];
+			if (programArguments.length == 2) {
+				outputPath = programArguments[1];
+			}
+		}
+
+		return true;
+	}
+
+	@SuppressWarnings("unchecked")
+	private static DataSet<Tuple> getDataSet(final ExecutionEnvironment env) {
+
+		DataSet<? extends Tuple> source;
+		if (filePath == null) {
+			source = env.fromCollection(getExampleInputTuples());
+
+		} else {
+			source = env
+					.readCsvFile(filePath)
+					.fieldDelimiter(';')
+					.types(String.class, String.class, String.class);
+
+		}
+
+		return (DataSet<Tuple>) source;
+	}
+
+	private static Collection<Tuple3<String, String, String>> getExampleInputTuples() {
+		Collection<Tuple3<String, String, String>> inputTuples = new ArrayList<Tuple3<String, String, String>>();
+		inputTuples.add(new Tuple3<String, String, String>("John", "Doe", "Foo Str."));
+		inputTuples.add(new Tuple3<String, String, String>("Joe", "Johnson", ""));
+		inputTuples.add(new Tuple3<String, String, String>(null, "Kate Morn", "Bar Blvd."));
+		inputTuples.add(new Tuple3<String, String, String>("Tim", "Rinny", ""));
+		inputTuples.add(new Tuple3<String, String, String>("Alicia", "Jackson", "  "));
+		return inputTuples;
+	}
+
+	/**
+	 * This function filters all incoming tuples that have one or more empty fields.
+	 * In doing so, it also counts the number of empty fields per attribute with an accumulator (registered under 
+	 * {@link EmptyFieldsCountAccumulator#EMPTY_FIELD_ACCUMULATOR}).
+	 */
+	public static final class EmptyFieldFilter extends RichFilterFunction<Tuple> {
+
+		// create a new accumulator in each filter function instance
+		// accumulators can be merged later on
+		private final VectorAccumulator emptyFieldCounter = new VectorAccumulator();
+
+		@Override
+		public void open(final Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			// register the accumulator instance
+			getRuntimeContext().addAccumulator(EMPTY_FIELD_ACCUMULATOR,
+					this.emptyFieldCounter);
+		}
+
+		@Override
+		public boolean filter(final Tuple t) {
+			boolean containsEmptyFields = false;
+
+			// iterate over the tuple fields looking for empty ones
+			for (int pos = 0; pos < t.getArity(); pos++) {
+
+				final String field = t.getField(pos);
+				if (field == null || field.trim().isEmpty()) {
+					containsEmptyFields = true;
+
+					// if an empty field is encountered, update the
+					// accumulator
+					this.emptyFieldCounter.add(pos);
+				}
+			}
+
+			return !containsEmptyFields;
+		}
+	}
+
+	/**
+	 * This accumulator lets you increase vector components distributedly. The {@link #add(Integer)} method lets you
+	 * increase the <i>n</i>-th vector component by 1, whereat <i>n</i> is the methods parameter. The size of the vector
+	 * is automatically managed.
+	 */
+	public static class VectorAccumulator implements Accumulator<Integer, List<Integer>> {
+
+		/** Stores the accumulated vector components. */
+		private final List<Integer> resultVector = new ArrayList<Integer>();
+
+		/**
+		 * Increases the result vector component at the specified position by 1.
+		 */
+		@Override
+		public void add(final Integer position) {
+			updateResultVector(position, 1);
+		}
+
+		/**
+		 * Increases the result vector component at the specified position by the specified delta.
+		 */
+		private void updateResultVector(final int position, final int delta) {
+			// inflate the vector to contain the given position
+			while (this.resultVector.size() <= position) {
+				this.resultVector.add(0);
+			}
+
+			// increment the component value
+			final int component = this.resultVector.get(position);
+			this.resultVector.set(position, component + delta);
+		}
+
+		@Override
+		public List<Integer> getLocalValue() {
+			return this.resultVector;
+		}
+
+		@Override
+		public void resetLocal() {
+			// clear the result vector if the accumulator instance shall be reused
+			this.resultVector.clear();
+		}
+
+		@Override
+		public void merge(final Accumulator<Integer, List<Integer>> other) {
+			// merge two vector accumulators by adding their up their vector components
+			final List<Integer> otherVector = other.getLocalValue();
+			for (int index = 0; index < otherVector.size(); index++) {
+				updateResultVector(index, otherVector.get(index));
+			}
+		}
+
+		@Override
+		public void write(final DataOutputView out) throws IOException {
+			// binary serialization of the result vector:
+			// [number of components, component 0, component 1, ...]
+			out.writeInt(this.resultVector.size());
+			for (final Integer component : this.resultVector) {
+				out.writeInt(component);
+			}
+		}
+
+		@Override
+		public void read(final DataInputView in) throws IOException {
+			// binary deserialization of the result vector
+			final int size = in.readInt();
+			for (int numReadComponents = 0; numReadComponents < size; numReadComponents++) {
+				final int component = in.readInt();
+				this.resultVector.add(component);
+			}
+		}
+
+	}
+}


[14/60] git commit: Move RichFunctions to api.common package

Posted by al...@apache.org.
Move RichFunctions to api.common package

They were in api.java before but they can be used from Scala,
just like the regular functions.


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

Branch: refs/heads/master
Commit: 568dff123d7ede05185be60842530076e234b3fa
Parents: a32890a
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Sep 3 14:50:42 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../flink/api/avro/AvroOutputFormatTest.java    |  2 +-
 .../avro/testjar/AvroExternalJarProgram.java    |  4 +-
 .../mapred/example/WordCount.java               |  4 +-
 .../mapreduce/example/WordCount.java            |  4 +-
 .../spargel/java/VertexCentricIteration.java    |  2 +-
 .../SpargelPageRankCountingVertices.java        |  2 +-
 .../SpargelConnectedComponentsITCase.java       |  2 +-
 .../connectors/json/JSONParseFlatMap.java       |  2 +-
 .../api/datastream/BatchedDataStream.java       |  4 +-
 .../api/datastream/ConnectedDataStream.java     |  2 +-
 .../streaming/api/datastream/DataStream.java    |  8 +-
 .../api/datastream/GroupedDataStream.java       |  2 +-
 .../apache/flink/streaming/api/IterateTest.java |  2 +-
 .../operator/GroupedBatchGroupReduceTest.java   |  2 +-
 .../streamcomponent/StreamComponentTest.java    |  2 +-
 .../serialization/TypeSerializationTest.java    |  2 +-
 .../ml/IncrementalLearningSkeleton.java         |  3 +
 .../apache/flink/client/testjar/WordCount.java  |  2 +-
 .../compiler/BranchingPlansCompilerTest.java    |  2 +-
 .../CachedMatchStrategyCompilerTest.java        |  2 +-
 .../compiler/CoGroupSolutionSetFirstTest.java   |  4 +-
 .../compiler/GroupReduceCompilationTest.java    |  2 +-
 .../flink/compiler/IterationsCompilerTest.java  |  8 +-
 .../flink/compiler/ReduceCompilationTest.java   |  2 +-
 .../compiler/UnionPropertyPropagationTest.java  |  2 +-
 .../WorksetIterationsJavaApiCompilerTest.java   |  6 +-
 .../testfunctions/DummyFlatJoinFunction.java    |  2 +-
 .../testfunctions/IdentityGroupReducer.java     |  4 +-
 .../compiler/testfunctions/IdentityMapper.java  |  2 +-
 .../testfunctions/SelectOneReducer.java         |  2 +-
 .../testfunctions/Top1GroupReducer.java         |  4 +-
 .../common/functions/RichCoGroupFunction.java   | 43 +++++++++
 .../api/common/functions/RichCrossFunction.java | 42 +++++++++
 .../common/functions/RichFilterFunction.java    | 39 +++++++++
 .../functions/RichFlatCombineFunction.java      | 41 +++++++++
 .../common/functions/RichFlatJoinFunction.java  | 42 +++++++++
 .../common/functions/RichFlatMapFunction.java   | 41 +++++++++
 .../functions/RichGroupReduceFunction.java      | 87 +++++++++++++++++++
 .../api/common/functions/RichJoinFunction.java  | 41 +++++++++
 .../api/common/functions/RichMapFunction.java   | 40 +++++++++
 .../functions/RichMapPartitionFunction.java     | 41 +++++++++
 .../common/functions/RichReduceFunction.java    | 38 ++++++++
 .../flink/example/java/clustering/KMeans.java   |  2 +-
 .../flink/example/java/ml/LinearRegression.java |  2 +-
 .../relational/EmptyFieldsCountAccumulator.java |  2 +-
 .../java/org/apache/flink/api/java/DataSet.java | 22 ++---
 .../api/java/functions/FlatMapIterator.java     |  3 +-
 .../api/java/functions/FunctionAnnotation.java  | 16 ++--
 .../api/java/functions/GroupReduceIterator.java |  1 +
 .../api/java/functions/RichCoGroupFunction.java | 43 ---------
 .../api/java/functions/RichCrossFunction.java   | 42 ---------
 .../api/java/functions/RichFilterFunction.java  | 39 ---------
 .../java/functions/RichFlatCombineFunction.java | 41 ---------
 .../java/functions/RichFlatJoinFunction.java    | 42 ---------
 .../api/java/functions/RichFlatMapFunction.java | 41 ---------
 .../java/functions/RichGroupReduceFunction.java | 91 --------------------
 .../api/java/functions/RichJoinFunction.java    | 41 ---------
 .../api/java/functions/RichMapFunction.java     | 40 ---------
 .../functions/RichMapPartitionFunction.java     | 41 ---------
 .../api/java/functions/RichReduceFunction.java  | 38 --------
 .../api/java/operators/AggregateOperator.java   |  4 +-
 .../api/java/operators/CoGroupOperator.java     |  6 +-
 .../api/java/operators/DistinctOperator.java    |  4 +-
 .../api/java/operators/GroupReduceOperator.java |  2 +-
 .../flink/api/java/operators/JoinOperator.java  |  8 +-
 .../java/operators/SingleInputUdfOperator.java  |  4 +-
 .../api/java/operators/SortedGrouping.java      |  4 +-
 .../api/java/operators/TwoInputUdfOperator.java |  4 +-
 .../flink/api/java/operators/UdfOperator.java   |  4 +-
 .../api/java/operators/UnsortedGrouping.java    |  8 +-
 .../translation/KeyExtractingMapper.java        |  2 +-
 .../translation/KeyRemovingMapper.java          |  2 +-
 .../PlanUnwrappingReduceGroupOperator.java      |  2 +-
 .../SemanticPropertiesTranslationTest.java      |  2 +
 .../DeltaIterationTranslationTest.java          |  6 +-
 .../translation/ReduceTranslationTests.java     |  2 +-
 .../java/type/extractor/TypeExtractorTest.java  | 12 +--
 .../lambdas/LambdaExtractionTest.java           |  2 +-
 .../runtime/operators/CachedMatchTaskTest.java  |  5 +-
 .../operators/CoGroupTaskExternalITCase.java    |  4 +-
 .../runtime/operators/CoGroupTaskTest.java      |  4 +-
 .../operators/CombineTaskExternalITCase.java    |  2 +-
 .../runtime/operators/CombineTaskTest.java      |  4 +-
 .../operators/ReduceTaskExternalITCase.java     |  4 +-
 .../flink/runtime/operators/ReduceTaskTest.java |  4 +-
 .../operators/chaining/ChainTaskTest.java       |  2 +-
 .../drivers/AllGroupReduceDriverTest.java       |  2 +-
 .../operators/drivers/AllReduceDriverTest.java  |  2 +-
 .../drivers/GroupReduceDriverTest.java          |  2 +-
 .../drivers/ReduceCombineDriverTest.java        |  2 +-
 .../operators/drivers/ReduceDriverTest.java     |  2 +-
 .../CombiningUnilateralSortMergerITCase.java    |  2 +-
 ...ultipleJoinsWithSolutionSetCompilerTest.java |  6 +-
 .../BulkIterationWithAllReducerITCase.java      |  2 +-
 .../CoGroupConnectedComponentsSecondITCase.java |  6 +-
 .../DependencyConnectedComponentsITCase.java    |  6 +-
 .../aggregators/AggregatorsITCase.java          |  4 +-
 ...nentsWithParametrizableAggregatorITCase.java |  4 +-
 ...entsWithParametrizableConvergenceITCase.java |  6 +-
 .../test/javaApiOperators/CoGroupITCase.java    |  2 +-
 .../test/javaApiOperators/CrossITCase.java      |  2 +-
 .../test/javaApiOperators/DistinctITCase.java   |  2 +-
 .../test/javaApiOperators/FilterITCase.java     |  2 +-
 .../test/javaApiOperators/FlatMapITCase.java    |  2 +-
 .../javaApiOperators/GroupReduceITCase.java     |  4 +-
 .../flink/test/javaApiOperators/JoinITCase.java |  2 +-
 .../flink/test/javaApiOperators/MapITCase.java  |  2 +-
 .../test/javaApiOperators/ReduceITCase.java     |  2 +-
 .../test/javaApiOperators/UnionITCase.java      |  2 +-
 .../flink/test/util/testjar/KMeansForTest.java  |  4 +-
 110 files changed, 652 insertions(+), 662 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java
index 386f318..7ae8df0 100644
--- a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java
+++ b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatTest.java
@@ -30,7 +30,7 @@ import org.apache.avro.reflect.ReflectDatumReader;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.io.AvroOutputFormat;
 import org.apache.flink.api.java.record.io.avro.example.User;
 import org.apache.flink.api.java.tuple.Tuple3;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
index 75b7da6..d0e52fd 100644
--- a/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
+++ b/flink-addons/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
@@ -40,8 +40,8 @@ import org.apache.avro.io.DatumWriter;
 import org.apache.avro.reflect.ReflectData;
 import org.apache.avro.reflect.ReflectDatumWriter;
 import org.apache.flink.api.avro.AvroBaseValue;
-import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java
index ba09e77..623b58d 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/example/WordCount.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.hadoopcompatibility.mapred.example;
 
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -29,7 +29,7 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.hadoopcompatibility.mapred.HadoopInputFormat;
 import org.apache.flink.hadoopcompatibility.mapred.HadoopOutputFormat;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java
index c00a14a..33ab97c 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/example/WordCount.java
@@ -19,8 +19,8 @@
 package org.apache.flink.hadoopcompatibility.mapreduce.example;
 
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.util.Collector;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
index 777cf9d..37cc549 100644
--- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
+++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
@@ -29,7 +29,7 @@ import org.apache.commons.lang3.Validate;
 import org.apache.flink.api.common.aggregators.Aggregator;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
 import org.apache.flink.api.java.operators.CoGroupOperator;
 import org.apache.flink.api.java.operators.CustomUnaryOperation;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java
index 01d2cd7..94946ef 100644
--- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java
+++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/examples/SpargelPageRankCountingVertices.java
@@ -21,7 +21,7 @@ package org.apache.flink.spargel.java.examples;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java b/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java
index 16b004c..948a708 100644
--- a/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java
+++ b/flink-addons/flink-spargel/src/test/java/org/apache/flink/test/spargel/SpargelConnectedComponentsITCase.java
@@ -21,7 +21,7 @@ package org.apache.flink.test.spargel;
 
 import java.io.BufferedReader;
 
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
index 96b1bf7..64f22ec 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.connectors.json;
 
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.sling.commons.json.JSONException;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
index 0249a1f..2565ce1 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
@@ -19,8 +19,8 @@ package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MaxAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MinAggregationFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index 9529dcd..6108eec 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -22,7 +22,7 @@ import java.io.Serializable;
 import org.apache.commons.lang3.SerializationException;
 import org.apache.commons.lang3.SerializationUtils;
 import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.function.co.CoFlatMapFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 0e1ae57..970415b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -28,10 +28,10 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichFilterFunction;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
index 8978b19..5261222 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
@@ -18,7 +18,7 @@
 package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.invokable.operator.GroupedReduceInvokable;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 83f7e8e..fc14256 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api;
 
 import static org.junit.Assert.assertTrue;
 
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.IterativeDataStream;
 import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
index 096141d..a640893 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
@@ -23,7 +23,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.util.MockInvokable;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index bfc8c7a..3e79401 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -25,7 +25,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
index c260077..cf89e2f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
@@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
 import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.junit.Test;
 
 public class TypeSerializationTest {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index 3218c47..d3cd2e6 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -134,4 +134,7 @@ public class IncrementalLearningSkeleton {
 
 		env.execute();
 	}
+
+
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java
index 2b64b84..2447602 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/WordCount.java
@@ -19,7 +19,7 @@
 package org.apache.flink.client.testjar;
 
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.util.Collector;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
index fd2f422..63ed907 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.api.java.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
 import org.apache.flink.api.java.record.operators.BulkIteration;
 import org.apache.flink.api.java.record.operators.CoGroupOperator;
 import org.apache.flink.api.java.record.operators.CrossOperator;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
index 1deead3..5c27b99 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.operators.base.GenericDataSourceBase;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.api.java.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.compiler.dag.TempMode;
 import org.apache.flink.compiler.plan.DualInputPlanNode;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
index 54af344..7c3fc27 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.compiler;
 
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java
index 9f63683..0441a6f 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/GroupReduceCompilationTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.compiler;
 
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.util.FieldList;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
index 562336f..3d423ef 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.compiler;
 
 import static org.junit.Assert.*;
 
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.junit.Test;
 
 import org.apache.flink.api.common.Plan;
@@ -29,9 +29,9 @@ import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichJoinFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java
index fb8ae8d..5a69b3c 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/ReduceCompilationTest.java
@@ -22,7 +22,7 @@ package org.apache.flink.compiler;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java
index 1020c8b..f4e5614 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/UnionPropertyPropagationTest.java
@@ -25,7 +25,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.base.FlatMapOperatorBase;
 import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.record.operators.FileDataSink;
 import org.apache.flink.api.java.record.operators.FileDataSource;
 import org.apache.flink.api.java.record.operators.ReduceOperator;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
index a7c6152..3e73565 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
@@ -29,9 +29,9 @@ import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichJoinFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.compiler.plan.DualInputPlanNode;
 import org.apache.flink.compiler.plan.OptimizedPlan;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java
index 2388db4..1cb3ac0 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/DummyFlatJoinFunction.java
@@ -19,7 +19,7 @@
 
 package org.apache.flink.compiler.testfunctions;
 
-import org.apache.flink.api.java.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
 import org.apache.flink.util.Collector;
 
 public class DummyFlatJoinFunction<T> extends RichFlatJoinFunction<T, T, T> {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java
index e06e3ba..4163e33 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityGroupReducer.java
@@ -19,8 +19,8 @@
 package org.apache.flink.compiler.testfunctions;
 
 
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable;
 import org.apache.flink.util.Collector;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java
index 29fc2c8..e797cf6 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityMapper.java
@@ -19,7 +19,7 @@
 
 package org.apache.flink.compiler.testfunctions;
 
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 
 public class IdentityMapper<T> extends RichMapFunction<T, T> {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java
index 7ce267f..71764dc 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/SelectOneReducer.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.compiler.testfunctions;
 
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 
 public class SelectOneReducer<T> extends RichReduceFunction<T> {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java
index b1a0e2d..f1b4e0b 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/Top1GroupReducer.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.compiler.testfunctions;
 
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable;
 import org.apache.flink.util.Collector;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java
new file mode 100644
index 0000000..b221cb7
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * Rich variant of the {@link CoGroupFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <IN1> The type of the elements in the first input.
+ * @param <IN2> The type of the elements in the second input.
+ * @param <OUT> The type of the result elements.
+ */
+public abstract class RichCoGroupFunction<IN1, IN2, OUT> extends AbstractRichFunction implements CoGroupFunction<IN1, IN2, OUT> {
+
+	private static final long serialVersionUID = 1L;
+	
+	@Override
+	public abstract void coGroup(Iterable<IN1> first, Iterable<IN2> second, Collector<OUT> out) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java
new file mode 100644
index 0000000..38ad961
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.CrossFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+
+/**
+ * Rich variant of the {@link CrossFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <IN1> The type of the elements in the first input.
+ * @param <IN2> The type of the elements in the second input.
+ * @param <OUT> The type of the result elements.
+ */
+public abstract class RichCrossFunction<IN1, IN2, OUT> extends AbstractRichFunction implements CrossFunction<IN1, IN2, OUT> {
+	
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract OUT cross(IN1 first, IN2 second) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java
new file mode 100644
index 0000000..b330866
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+
+/**
+ * Rich variant of the {@link FilterFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <T> The type of the filtered elements.
+ */
+public abstract class RichFilterFunction<T> extends AbstractRichFunction implements FilterFunction<T> {
+	
+	private static final long serialVersionUID = 1L;
+	
+	@Override
+	public abstract boolean filter(T value) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatCombineFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatCombineFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatCombineFunction.java
new file mode 100644
index 0000000..5d6717d
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatCombineFunction.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FlatCombineFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * Rich variant of the {@link FlatCombineFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ *
+ * @param <T> The data type of the elements to be combined.
+ */
+public abstract class RichFlatCombineFunction<T> extends AbstractRichFunction implements FlatCombineFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract void combine(Iterable<T> values, Collector<T> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java
new file mode 100644
index 0000000..e1ec0f1
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * Rich variant of the {@link FlatJoinFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <IN1> The type of the elements in the first input.
+ * @param <IN2> The type of the elements in the second input.
+ * @param <OUT> The type of the result elements.
+ */
+public abstract class RichFlatJoinFunction<IN1, IN2, OUT> extends AbstractRichFunction implements FlatJoinFunction<IN1, IN2, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract void join(IN1 first, IN2 second, Collector<OUT> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java
new file mode 100644
index 0000000..f3ec4a7
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * Rich variant of the {@link FlatMapFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <IN> Type of the input elements.
+ * @param <OUT> Type of the returned elements.
+ */
+public abstract class RichFlatMapFunction<IN, OUT> extends AbstractRichFunction implements FlatMapFunction<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract void flatMap(IN value, Collector<OUT> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
new file mode 100644
index 0000000..6c7edff
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.flink.util.Collector;
+
+/**
+ * Rich variant of the {@link GroupReduceFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <IN> Type of the elements that this function processes.
+ * @param <OUT> The type of the elements returned by the user-defined function.
+ */
+public abstract class RichGroupReduceFunction<IN, OUT> extends AbstractRichFunction implements GroupReduceFunction<IN, OUT>, FlatCombineFunction<IN> {
+	
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract void reduce(Iterable<IN> values, Collector<OUT> out) throws Exception;
+	
+	/**
+	 * The combine methods pre-reduces elements. It may be called on subsets of the data
+	 * before the actual reduce function. This is often helpful to lower data volume prior
+	 * to reorganizing the data in an expensive way, as might be required for the final
+	 * reduce function.
+	 * <p>
+	 * This method is only ever invoked when the subclass of {@link RichGroupReduceFunction}
+	 * adds the {@link Combinable} annotation, or if the <i>combinable</i> flag is set when defining
+	 * the <i>reduceGroup<i> operation via
+	 * {@link org.apache.flink.api.java.operators.GroupReduceOperator#setCombinable(boolean)}.
+	 * <p>
+	 * Since the reduce function will be called on the result of this method, it is important that this
+	 * method returns the same data type as it consumes. By default, this method only calls the
+	 * {@link #reduce(Iterable, Collector)} method. If the behavior in the pre-reducing is different
+	 * from the final reduce function (for example because the reduce function changes the data type),
+	 * this method must be overwritten, or the execution will fail.
+	 * 
+	 * @param values The iterator returning the group of values to be reduced.
+	 * @param out The collector to emit the returned values.
+	 * 
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	@Override
+	public void combine(Iterable<IN> values, Collector<IN> out) throws Exception {
+		@SuppressWarnings("unchecked")
+		Collector<OUT> c = (Collector<OUT>) out;
+		reduce(values, c);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * This annotation can be added to classes that extend {@link RichGroupReduceFunction}, in oder to mark
+	 * them as "combinable". The system may call the {@link RichGroupReduceFunction#combine(Iterable, Collector)}
+	 * method on such functions, to pre-reduce the data before transferring it over the network to
+	 * the actual group reduce operation.
+	 * <p>
+	 * Marking combinable functions as such is in general beneficial for performance.
+	 */
+	@Retention(RetentionPolicy.RUNTIME)
+	@Target(ElementType.TYPE)
+	public static @interface Combinable {};
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java
new file mode 100644
index 0000000..9139fc4
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+
+/**
+ * Rich variant of the {@link JoinFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ *
+ * @param <IN1> The type of the elements in the first input.
+ * @param <IN2> The type of the elements in the second input.
+ * @param <OUT> The type of the result elements.
+ */
+public abstract class RichJoinFunction<IN1,IN2,OUT> extends AbstractRichFunction implements JoinFunction<IN1,IN2,OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract OUT join(IN1 first, IN2 second) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java
new file mode 100644
index 0000000..30b6666
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+
+/**
+ * Rich variant of the {@link MapFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <IN> Type of the input elements.
+ * @param <OUT> Type of the returned elements.
+ */
+public abstract class RichMapFunction<IN, OUT> extends AbstractRichFunction implements MapFunction<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public abstract OUT map(IN value) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java
new file mode 100644
index 0000000..7fce2a8
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * Rich variant of the {@link MapPartitionFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <I> Type of the input elements.
+ * @param <O> Type of the returned elements.
+ */
+public abstract class RichMapPartitionFunction<I, O> extends AbstractRichFunction implements MapPartitionFunction<I, O> {
+
+	private static final long serialVersionUID = 1L;
+	
+	@Override
+	public abstract void mapPartition(Iterable<I> values, Collector<O> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java
new file mode 100644
index 0000000..c630510
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichFunction;
+
+/**
+ * Rich variant of the {@link ReduceFunction}. As a {@link RichFunction}, it gives access to the
+ * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
+ * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
+ * {@link RichFunction#close()}.
+ * 
+ * @param <T> Type of the elements that this function processes.
+ */
+public abstract class RichReduceFunction<T> extends AbstractRichFunction implements ReduceFunction<T> {
+	
+	private static final long serialVersionUID = 1L;
+
+	public abstract T reduce(T value1, T value2) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
index 3c31af6..3bea458 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
@@ -23,7 +23,7 @@ import java.util.Collection;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
index d65a809..6ef6270 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
@@ -24,7 +24,7 @@ import java.util.Collection;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.example.java.ml.util.LinearRegressionData;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
index 4bced17..a379bf8 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
@@ -26,7 +26,7 @@ import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
index 3dcd780..4c111ff 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
@@ -132,13 +132,13 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Applies a Map transformation on a {@link DataSet}.<br/>
-	 * The transformation calls a {@link org.apache.flink.api.java.functions.RichMapFunction} for each element of the DataSet.
+	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichMapFunction} for each element of the DataSet.
 	 * Each MapFunction call returns exactly one element.
 	 * 
 	 * @param mapper The MapFunction that is called for each element of the DataSet.
 	 * @return A MapOperator that represents the transformed DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichMapFunction
+	 * @see org.apache.flink.api.common.functions.RichMapFunction
 	 * @see MapOperator
 	 * @see DataSet
 	 */
@@ -185,13 +185,13 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Applies a FlatMap transformation on a {@link DataSet}.<br/>
-	 * The transformation calls a {@link org.apache.flink.api.java.functions.RichFlatMapFunction} for each element of the DataSet.
+	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichFlatMapFunction} for each element of the DataSet.
 	 * Each FlatMapFunction call can return any number of elements including none.
 	 * 
 	 * @param flatMapper The FlatMapFunction that is called for each element of the DataSet. 
 	 * @return A FlatMapOperator that represents the transformed DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichFlatMapFunction
+	 * @see org.apache.flink.api.common.functions.RichFlatMapFunction
 	 * @see FlatMapOperator
 	 * @see DataSet
 	 */
@@ -208,14 +208,14 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Applies a Filter transformation on a {@link DataSet}.<br/>
-	 * The transformation calls a {@link org.apache.flink.api.java.functions.RichFilterFunction} for each element of the DataSet
+	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichFilterFunction} for each element of the DataSet
 	 * and retains only those element for which the function returns true. Elements for 
 	 * which the function returns false are filtered. 
 	 * 
 	 * @param filter The FilterFunction that is called for each element of the DataSet.
 	 * @return A FilterOperator that represents the filtered DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichFilterFunction
+	 * @see org.apache.flink.api.common.functions.RichFilterFunction
 	 * @see FilterOperator
 	 * @see DataSet
 	 */
@@ -311,14 +311,14 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Applies a Reduce transformation on a non-grouped {@link DataSet}.<br/>
-	 * The transformation consecutively calls a {@link org.apache.flink.api.java.functions.RichReduceFunction}
+	 * The transformation consecutively calls a {@link org.apache.flink.api.common.functions.RichReduceFunction}
 	 *   until only a single element remains which is the result of the transformation.
 	 * A ReduceFunction combines two elements into one new element of the same type.
 	 * 
 	 * @param reducer The ReduceFunction that is applied on the DataSet.
 	 * @return A ReduceOperator that represents the reduced DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichReduceFunction
+	 * @see org.apache.flink.api.common.functions.RichReduceFunction
 	 * @see ReduceOperator
 	 * @see DataSet
 	 */
@@ -331,14 +331,14 @@ public abstract class DataSet<T> {
 	
 	/**
 	 * Applies a GroupReduce transformation on a non-grouped {@link DataSet}.<br/>
-	 * The transformation calls a {@link org.apache.flink.api.java.functions.RichGroupReduceFunction} once with the full DataSet.
+	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} once with the full DataSet.
 	 * The GroupReduceFunction can iterate over all elements of the DataSet and emit any
 	 *   number of output elements including none.
 	 * 
 	 * @param reducer The GroupReduceFunction that is applied on the DataSet.
 	 * @return A GroupReduceOperator that represents the reduced DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichGroupReduceFunction
+	 * @see org.apache.flink.api.common.functions.RichGroupReduceFunction
 	 * @see org.apache.flink.api.java.operators.GroupReduceOperator
 	 * @see DataSet
 	 */
@@ -600,7 +600,7 @@ public abstract class DataSet<T> {
 	 * Initiates a CoGroup transformation.<br/>
 	 * A CoGroup transformation combines the elements of
 	 *   two {@link DataSet DataSets} into one DataSet. It groups each DataSet individually on a key and 
-	 *   gives groups of both DataSets with equal keys together into a {@link org.apache.flink.api.java.functions.RichCoGroupFunction}.
+	 *   gives groups of both DataSets with equal keys together into a {@link org.apache.flink.api.common.functions.RichCoGroupFunction}.
 	 *   If a DataSet has a group with no matching key in the other DataSet, the CoGroupFunction
 	 *   is called with an empty group for the non-existing group.</br>
 	 * The CoGroupFunction can iterate over the elements of both groups and return any number 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java
index 0a83235..2030de3 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FlatMapIterator.java
@@ -20,10 +20,11 @@ package org.apache.flink.api.java.functions;
 
 import java.util.Iterator;
 
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.util.Collector;
 
 /**
- * A convenience variant of the {@link RichFlatMapFunction} that returns elements through an iterator, rather then
+ * A convenience variant of the {@link org.apache.flink.api.common.functions.RichFlatMapFunction} that returns elements through an iterator, rather then
  * through a collector. In all other respects, it behaves exactly like the FlatMapFunction.
  * <p>
  * The function needs to be serializable, as defined in {@link java.io.Serializable}.


[30/60] git commit: Connected Components example Scala API rewrite

Posted by al...@apache.org.
Connected Components example Scala API rewrite


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

Branch: refs/heads/master
Commit: 84b04bebacfc064b5cd0715328cdcd43cf4e5d0a
Parents: b8131fa
Author: vasia <va...@gmail.com>
Authored: Tue Sep 9 11:16:59 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../graph/util/ConnectedComponentsData.java     |  59 ++--
 .../scala/graph/ConnectedComponents.scala       | 275 +++++++++++++------
 2 files changed, 220 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/84b04beb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
index ddc6eff..aa5bbd6 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/ConnectedComponentsData.java
@@ -19,11 +19,10 @@
 
 package org.apache.flink.example.java.graph.util;
 
-import java.util.ArrayList;
+import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.flink.api.java.tuple.Tuple2;
-
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 
@@ -33,34 +32,46 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  *
  */
 public class ConnectedComponentsData {
+	
+	public static final Object[][] VERTICES  = new Object[][] {
+		new Object[]{1L}, new Object[]{2L}, new Object[]{3L}, new Object[]{4L}, 
+		new Object[]{5L},new Object[]{6L}, new Object[]{7L}, new Object[]{8L}, 
+		new Object[]{9L}, new Object[]{10L}, new Object[]{11L}, new Object[]{12L}, 
+		new Object[]{13L}, new Object[]{14L}, new Object[]{15L}, new Object[]{16L}
+	};
 
 	public static DataSet<Long> getDefaultVertexDataSet(ExecutionEnvironment env) {
-		
-		return env.fromElements(
-				1L, 2L, 3L, 4L, 5L, 
-				6L, 7L, 8L, 9L, 10L,
-				11L, 12L, 13L, 14L, 15L, 16L);
+		List<Long> verticesList = new LinkedList<Long>();
+		for (Object[] vertex : VERTICES) {
+			verticesList.add((Long) vertex[0]);
+		}
+		return env.fromCollection(verticesList);
 	}
 	
+	public static final Object[][] EDGES = new Object[][] {
+		new Object[]{1L, 2L},
+		new Object[]{2L, 3L},
+		new Object[]{2L, 4L},
+		new Object[]{3L, 5L},
+		new Object[]{6L, 7L},
+		new Object[]{8L, 9L},
+		new Object[]{8L, 10L},
+		new Object[]{5L, 11L},
+		new Object[]{11L, 12L},
+		new Object[]{10L, 13L},
+		new Object[]{9L, 14L},
+		new Object[]{13L, 14L},
+		new Object[]{1L, 15L},
+		new Object[]{16L, 1L}
+	};
+	
 	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 		
-		List<Tuple2<Long, Long>> data = new ArrayList<Tuple2<Long, Long>>();
-		data.add(new Tuple2<Long, Long>(1L, 2L));
-		data.add(new Tuple2<Long, Long>(2L, 3L));
-		data.add(new Tuple2<Long, Long>(2L, 4L));
-		data.add(new Tuple2<Long, Long>(3L, 5L));
-		data.add(new Tuple2<Long, Long>(6L, 7L));
-		data.add(new Tuple2<Long, Long>(8L, 9L));
-		data.add(new Tuple2<Long, Long>(8L, 10L));
-		data.add(new Tuple2<Long, Long>(5L, 11L));
-		data.add(new Tuple2<Long, Long>(11L, 12L));
-		data.add(new Tuple2<Long, Long>(10L, 13L));
-		data.add(new Tuple2<Long, Long>(9L, 14L));
-		data.add(new Tuple2<Long, Long>(13L, 14L));
-		data.add(new Tuple2<Long, Long>(1L, 15L));
-		data.add(new Tuple2<Long, Long>(16L, 1L));
-		
-		return env.fromCollection(data);
+		List<Tuple2<Long, Long>> edgeList = new LinkedList<Tuple2<Long, Long>>();
+		for (Object[] edge : EDGES) {
+			edgeList.add(new Tuple2<Long, Long>((Long) edge[0], (Long) edge[1]));
+		}
+		return env.fromCollection(edgeList);
 	}
 	
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/84b04beb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
index 696b75f..f440b6f 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
@@ -1,90 +1,185 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.graph;
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//object RunConnectedComponents {
-// def main(pArgs: Array[String]) {
-//
-//    if (pArgs.size < 5) {
-//      println("USAGE: <vertices input file> <edges input file> <output file> <max iterations> <degree of parallelism>")
-//      return
-//    }
-//    val plan = new ConnectedComponents().getPlan(pArgs(0), pArgs(1), pArgs(2), pArgs(3), pArgs(4))
-//    LocalExecutor.execute(plan)
-//  }
-//}
-//
-//class ConnectedComponents extends Program with Serializable {
-//
-//    override def getPlan(args: String*) = {
-//      val plan = getScalaPlan(args(0), args(1), args(2), args(3).toInt)
-//      plan.setDefaultParallelism(args(4).toInt)
-//      plan
-//  }
-//
-//  def getScalaPlan(verticesInput: String, edgesInput: String, componentsOutput: String, maxIterations: Int) = {
-//
-//  val vertices = DataSource(verticesInput, DelimitedInputFormat(parseVertex))
-//  val directedEdges = DataSource(edgesInput, DelimitedInputFormat(parseEdge))
-//
-//  val undirectedEdges = directedEdges flatMap { case (from, to) => Seq(from -> to, to -> from) }
-//
-//    def propagateComponent(s: DataSetOLD[(Int, Int)], ws: DataSetOLD[(Int, Int)]) = {
-//
-//      val allNeighbors = ws join undirectedEdges where { case (v, _) => v } isEqualTo { case (from, _) => from } map { (w, e) => e._2 -> w._2 }
-//      val minNeighbors = allNeighbors groupBy { case (to, _) => to } reduceGroup { cs => cs minBy { _._2 } }
-//
-//      // updated solution elements == new workset
-//      val s1 = s join minNeighbors where { _._1 } isEqualTo { _._1 } flatMap { (n, s) =>
-//        (n, s) match {
-//          case ((v, cOld), (_, cNew)) if cNew < cOld => Some((v, cNew))
-//          case _ => None
-//        }
-//      }
-////      s1.left preserves({ case (v, _) => v }, { case (v, _) => v })
-//      s1.right preserves({ v=>v }, { v=>v })
-//
-//      (s1, s1)
-//    }
-//
-//    val components = vertices.iterateWithDelta(vertices, { _._1 }, propagateComponent, maxIterations)
-//    val output = components.write(componentsOutput, DelimitedOutputFormat(formatOutput.tupled))
-//
-//    val plan = new ScalaPlan(Seq(output), "Connected Components")
-//    plan
-//  }
-//
-//  def parseVertex = (line: String) => { val v = line.toInt; v -> v }
-//
-//  val EdgeInputPattern = """(\d+) (\d+)""".r
-//
-//  def parseEdge = (line: String) => line match {
-//    case EdgeInputPattern(from, to) => from.toInt -> to.toInt
-//  }
-//
-//  def formatOutput = (vertex: Int, component: Int) => "%d %d".format(vertex, component)
-//}
-//
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.graph;
+
+import org.apache.flink.api.scala._
+import org.apache.flink.example.java.graph.util.ConnectedComponentsData
+import org.apache.flink.api.java.aggregation.Aggregations
+import org.apache.flink.util.Collector
+
+/**
+ * An implementation of the connected components algorithm, using a delta iteration.
+ * 
+ * Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the minimum of its own ID and its
+ * neighbors' IDs, as its new ID and tells its neighbors about its new ID. After the algorithm has completed, all vertices in the
+ * same component will have the same ID.
+ * 
+ * A vertex whose component ID did not change needs not propagate its information in the next step. Because of that,
+ * the algorithm is easily expressible via a delta iteration. We here model the solution set as the vertices with
+ * their current component ids, and the workset as the changed vertices. Because we see all vertices initially as
+ * changed, the initial workset and the initial solution set are identical. Also, the delta to the solution set
+ * is consequently also the next workset.
+ * 
+ * Input files are plain text files and must be formatted as follows:
+ *
+ *   - Vertices represented as IDs and separated by new-line characters.
+ *    
+ * For example 
+ *  {{{
+ *  "1\n2\n12\n42\n63\n"
+ *  }}}
+ *  gives five vertices (1), (2), (12), (42), and (63).
+ *  
+ *  
+ *   - Edges are represented as pairs for vertex IDs which are separated by space 
+ * characters. Edges are separated by new-line characters.
+ * 
+ * For example 
+ *  {{{
+ *  "1 2\n2 12\n1 12\n42 63\n"
+ *   }}}
+ *   gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+ *
+ * 
+ *
+ * Usage:
+ *  {{{ 
+ * 		ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>
+ *   }}}
+ *   
+ * If no parameters are provided, the program is run with default data from
+ *  [[org.apache.flink.example.java.graph.util.ConnectedComponentsData]]
+ *  and 10 iterations. 
+ * 
+ *
+ * This example shows how to use:
+ *
+ *   - Delta Iterations
+ *   - Generic-typed Functions 
+ *   
+ */
+object ConnectedComponents {
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    // set up execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    // read vertex and edge data 
+    // assign the initial components (equal to the vertex id)
+    val vertices = getVerticesDataSet(env).map(id => (id, id))
+    
+    // undirected edges by emitting for each input edge the input edges itself and an inverted version
+    val edges = getEdgesDataSet(env)
+      .flatMap { (edge, out: Collector[(Long, Long)]) => 
+      out.collect(edge)
+      out.collect((edge._2, edge._1))
+      }
+    
+    // open a delta iteration
+    val verticesWithComponents = vertices.iterateDelta(vertices, maxIterations, Array(0)) {
+      (s, ws) => {
+
+        // apply the step logic: join with the edges 
+        val allNeighbors = ws.join(edges)
+          .where(0).equalTo(0)
+          .map { in => (in._2._2, in._1._2) }
+        
+        // select the minimum neighbor 
+        val minNeighbors = allNeighbors.groupBy(0).aggregate(Aggregations.MIN, 1)
+        
+        // update if the component of the candidate is smaller
+        val updatedComponents = minNeighbors.join(s).where(0).equalTo(0)
+        		.flatMap  { newAndOldComponent => 
+        		  newAndOldComponent match {
+        		    case ((vId, cNew), (_, cOld)) if cNew < cOld => Some((vId, cNew))
+        		    case _ => None 
+        		    }
+        		  }
+        // delta and new workset are identical
+       (updatedComponents, updatedComponents)
+      }
+    }
+      if (fileOutput) {
+      verticesWithComponents.writeAsCsv(outputPath, "\n", " ")
+    } else {
+      verticesWithComponents.print()
+    }
+
+    env.execute("Scala Connected Components Example")
+  }
+ 
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+        fileOutput = true
+      if (args.length == 4) {
+        verticesPath = args(0)
+        edgesPath = args(1)
+        outputPath = args(2)
+        maxIterations = args(3).toInt
+      } else {
+        System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>")
+        false
+      }
+    } else {
+      System.out.println("Executing Connected Components example with built-in default data.")
+      System.out.println("  Provide parameters to read input data from a file.")
+      System.out.println("  Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>")
+    }
+    true
+  }
+
+  private def getVerticesDataSet(env: ExecutionEnvironment): DataSet[Long] = {
+    if (fileOutput) {
+       env.readCsvFile[Tuple1[Long]](
+        verticesPath,
+        includedFields = Array(0))
+        .map { x => x._1 }
+    }
+    else {
+      val vertexData = ConnectedComponentsData.VERTICES map {
+        case Array(x) => x.asInstanceOf[Long]
+      }
+      env.fromCollection(vertexData);      
+    }
+  }
+  
+  private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = {
+    if (fileOutput) {
+      env.readCsvFile[(Long, Long)](
+        edgesPath,
+        fieldDelimiter = ' ',
+        includedFields = Array(0, 1))
+        .map { x => (x._1, x._2)}
+    }
+    else {
+      val edgeData = ConnectedComponentsData.EDGES map {
+        case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
+      }
+      env.fromCollection(edgeData);
+    }
+  }
+
+  private var fileOutput: Boolean = false
+  private var verticesPath: String = null
+  private var edgesPath: String = null
+  private var maxIterations: Int = 10
+  private var outputPath: String = null
+}
\ No newline at end of file


[08/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 11ee48f..041f269 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -16,57 +16,822 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.scala
 
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import org.apache.flink.api.scala.operators._
-
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.types.Record
-
-
-class DataSet[T] (val contract: Operator[Record] with ScalaOperator[T, Record]) {
-  
-  def cogroup[RightIn](rightInput: DataSet[RightIn]) =
-    new CoGroupDataSet[T, RightIn](this, rightInput)
-  def cross[RightIn](rightInput: DataSet[RightIn]) = new CrossDataSet[T, RightIn](this, rightInput)
-  def join[RightIn](rightInput: DataSet[RightIn]) = new JoinDataSet[T, RightIn](this, rightInput)
-  
-  def map[Out](fun: T => Out) = macro MapMacros.map[T, Out]
-  def flatMap[Out](fun: T => Iterator[Out]) = macro MapMacros.flatMap[T, Out]
-  def filter(fun: T => Boolean) = macro MapMacros.filter[T]
-  
-  // reduce
-  def groupBy[Key](keyFun: T => Key) = macro ReduceMacros.groupBy[T, Key]
-
-  // reduce without grouping
-  def reduce(fun: (T, T) => T) = macro ReduceMacros.globalReduce[T]
-  def reduceAll[Out](fun: Iterator[T] => Out) = macro ReduceMacros.globalReduceGroup[T, Out]
-  def combinableReduceAll[Out](fun: Iterator[T] => Out) =
-    macro ReduceMacros.combinableGlobalReduceGroup[T]
-
-  def union(secondInput: DataSet[T]) = UnionOperator.impl[T](this, secondInput)
-  
-  def iterateWithDelta[DeltaItem](stepFunction: DataSet[T] => (DataSet[T], DataSet[DeltaItem])) =
-    macro IterateMacros.iterateWithDelta[T, DeltaItem]
-  def iterate(n: Int, stepFunction: DataSet[T] => DataSet[T])= macro IterateMacros.iterate[T]
-  def iterateWithTermination[C](
-      n: Int,
-      stepFunction: DataSet[T] => DataSet[T],
-      terminationFunction: (DataSet[T],DataSet[T]) => DataSet[C]) =
-    macro IterateMacros.iterateWithTermination[T, C]
-  def iterateWithDelta[SolutionKey, WorksetItem](
-      workset: DataSet[WorksetItem],
-      solutionSetKey: T => SolutionKey,
-      stepFunction: (DataSet[T], DataSet[WorksetItem]) =>
-                    (DataSet[T], DataSet[WorksetItem]), maxIterations: Int) =
-    macro WorksetIterateMacros.iterateWithDelta[T, SolutionKey, WorksetItem]
-  
-  def write(url: String, format: ScalaOutputFormat[T]) = DataSinkOperator.write(this, url, format)
-  def write(url: String, format: ScalaOutputFormat[T], name: String) =
-    DataSinkOperator.write(this, url, format, name)
-  
+import org.apache.commons.lang3.Validate
+import org.apache.flink.api.common.aggregators.Aggregator
+import org.apache.flink.api.common.functions._
+import org.apache.flink.api.common.io.{FileOutputFormat, OutputFormat}
+import org.apache.flink.api.java.aggregation.Aggregations
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.io.{PrintingOutputFormat, TextOutputFormat}
+import org.apache.flink.api.java.operators.JoinOperator.JoinHint
+import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
+import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+import org.apache.flink.api.scala.operators.{ScalaCsvOutputFormat, ScalaAggregateOperator}
+import org.apache.flink.core.fs.FileSystem.WriteMode
+import org.apache.flink.core.fs.{FileSystem, Path}
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.Collector
+
+import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
+
+/**
+ * The DataSet, the basic abstraction of Flink. This represents a collection of elements of a
+ * specific type `T`. The operations in this class can be used to create new DataSets and to combine
+ * two DataSets. The methods of [[ExecutionEnvironment]] can be used to create a DataSet from an
+ * external source, such as files in HDFS. The `write*` methods can be used to write the elements
+ * to storage.
+ *
+ * All operations accept either a lambda function or an operation-specific function object for
+ * specifying the operation. For example, using a lambda:
+ * {{{
+ *   val input: DataSet[String] = ...
+ *   val mapped = input flatMap { _.split(" ") }
+ * }}}
+ * And using a `MapFunction`:
+ * {{{
+ *   val input: DataSet[String] = ...
+ *   val mapped = input flatMap { new FlatMapFunction[String, String] {
+ *     def flatMap(in: String, out: Collector[String]): Unit = {
+ *       in.split(" ") foreach { out.collect(_) }
+ *     }
+ *   }
+ * }}}
+ *
+ * A rich function can be used when more control is required, for example for accessing the
+ * `RuntimeContext`. The rich function for `flatMap` is `RichFlatMapFunction`, all other functions
+ * are named similarly. All functions are available in package
+ * `org.apache.flink.api.common.functions`.
+ *
+ * The elements are partitioned depending on the degree of parallelism of the
+ * [[ExecutionEnvironment]] or of one specific DataSet.
+ *
+ * Most of the operations have an implicit [[TypeInformation]] parameter. This is supplied by
+ * an implicit conversion in the `flink.api.scala` Package. For this to work,
+ * [[createTypeInformation]] needs to be imported. This is normally achieved with a
+ * {{{
+ *   import org.apache.flink.api.scala._
+ * }}}
+ *
+ * @tparam T The type of the DataSet, i.e., the type of the elements of the DataSet.
+ */
+class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
+  Validate.notNull(set, "Java DataSet must not be null.")
+
+  // --------------------------------------------------------------------------------------------
+  //  General methods
+  // --------------------------------------------------------------------------------------------
+  // These are actually implemented in subclasses of the Java DataSet but we perform checking
+  // here and just pass through the calls to make everything much simpler.
+
+  /**
+   * Sets the name of the DataSet. This will appear in logs and graphical
+   * representations of the execution graph.
+   */
+  def name(name: String) = {
+    set match {
+      case ds: DataSource[_] => ds.name(name)
+      case op: Operator[_, _] => op.name(name)
+      case di: DeltaIterationResultSet[_, _] => di.getIterationHead.name(name)
+      case _ =>
+        throw new UnsupportedOperationException("Operator " + set.toString + " cannot have a name.")
+    }
+    // return this for chaining methods calls
+    this
+  }
+
+  /**
+   * Sets the degree of parallelism of this operation. This must be greater than 1.
+   */
+  def setParallelism(dop: Int) = {
+    set match {
+      case ds: DataSource[_] => ds.setParallelism(dop)
+      case op: Operator[_, _] => op.setParallelism(dop)
+      case di: DeltaIterationResultSet[_, _] => di.getIterationHead.parallelism(dop)
+      case _ =>
+        throw new UnsupportedOperationException("Operator " + set.toString + " cannot have " +
+          "parallelism.")
+    }
+    this
+  }
+
+  /**
+   * Returns the degree of parallelism of this operation.
+   */
+  def getParallelism: Int = set match {
+    case ds: DataSource[_] => ds.getParallelism
+    case op: Operator[_, _] => op.getParallelism
+    case _ =>
+      throw new UnsupportedOperationException("Operator " + set.toString + " does not have " +
+        "parallelism.")
+  }
+
+  /**
+   * Registers an [[org.apache.flink.api.common.aggregators.Aggregator]]
+   * for the iteration. Aggregators can be used to maintain simple statistics during the
+   * iteration, such as number of elements processed. The aggregators compute global aggregates:
+   * After each iteration step, the values are globally aggregated to produce one aggregate that
+   * represents statistics across all parallel instances.
+   * The value of an aggregator can be accessed in the next iteration.
+   *
+   * Aggregators can be accessed inside a function via
+   * [[org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext]].
+   *
+   * @param name The name under which the aggregator is registered.
+   * @param aggregator The aggregator class.
+   */
+  def registerAggregator(name: String, aggregator: Aggregator[_]): DataSet[T] = {
+    set match {
+      case di: DeltaIterationResultSet[_, _] =>
+        di.getIterationHead.registerAggregator(name, aggregator)
+      case _ =>
+        throw new UnsupportedOperationException("Operator " + set.toString + " cannot have " +
+          "aggregators.")
+    }
+    this
+  }
+
+  /**
+   * Adds a certain data set as a broadcast set to this operator. Broadcast data sets are
+   * available at all
+   * parallel instances of this operator. A broadcast data set is registered under a certain
+   * name, and can be
+   * retrieved under that name from the operators runtime context via
+   * `org.apache.flink.api.common.functions.RuntimeContext.getBroadCastVariable(String)`
+   *
+   * The runtime context itself is available in all UDFs via
+   * `org.apache.flink.api.common.functions.AbstractRichFunction#getRuntimeContext()`
+   *
+   * @param data The data set to be broadcasted.
+   * @param name The name under which the broadcast data set retrieved.
+   * @return The operator itself, to allow chaining function calls.
+   */
+  def withBroadcastSet(data: DataSet[_], name: String) = {
+    set match {
+      case udfOp: UdfOperator[_] => udfOp.withBroadcastSet(data.set, name)
+      case _ =>
+        throw new UnsupportedOperationException("Operator " + set.toString + " cannot have " +
+          "broadcast variables.")
+    }
+    this
+  }
+
+  def withConstantSet(constantSets: String*) = {
+    set match {
+      case op: SingleInputUdfOperator[_, _, _] => op.withConstantSet(constantSets: _*)
+      case _ =>
+        throw new UnsupportedOperationException("Cannot specify constant sets on Operator " +
+          set.toString + ".")
+    }
+    this
+  }
+
+  def withConstantSetFirst(constantSets: String*) = {
+    set match {
+      case op: TwoInputUdfOperator[_, _, _, _] => op.withConstantSetFirst(constantSets: _*)
+      case _ =>
+        throw new UnsupportedOperationException("Cannot specify constant sets on Operator " + set
+          .toString + ".")
+    }
+    this
+  }
+
+  def withConstantSetSecond(constantSets: String*) = {
+    set match {
+      case op: TwoInputUdfOperator[_, _, _, _] => op.withConstantSetSecond(constantSets: _*)
+      case _ =>
+        throw new UnsupportedOperationException("Cannot specify constant sets on Operator " + set
+          .toString + ".")
+    }
+    this
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Filter & Transformations
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new DataSet by applying the given function to every element of this DataSet.
+   */
+  def map[R: TypeInformation: ClassTag](mapper: MapFunction[T, R]): DataSet[R] = {
+    if (mapper == null) {
+      throw new NullPointerException("Map function must not be null.")
+    }
+    wrap(new MapOperator[T, R](set, implicitly[TypeInformation[R]], mapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to every element of this DataSet.
+   */
+  def map[R: TypeInformation: ClassTag](fun: (T) => R): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("Map function must not be null.")
+    }
+    val mapper = new MapFunction[T, R] {
+      def map(in: T): R = fun(in)
+    }
+    wrap(new MapOperator[T, R](set, implicitly[TypeInformation[R]], mapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to each parallel partition of the
+   * DataSet.
+   *
+   * This function is intended for operations that cannot transform individual elements and
+   * requires no grouping of elements. To transform individual elements,
+   * the use of [[map]] and [[flatMap]] is preferable.
+   */
+  def mapPartition[R: TypeInformation: ClassTag](
+      partitionMapper: MapPartitionFunction[T, R]): DataSet[R] = {
+    if (partitionMapper == null) {
+      throw new NullPointerException("MapPartition function must not be null.")
+    }
+    wrap(new MapPartitionOperator[T, R](set, implicitly[TypeInformation[R]], partitionMapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to each parallel partition of the
+   * DataSet.
+   *
+   * This function is intended for operations that cannot transform individual elements and
+   * requires no grouping of elements. To transform individual elements,
+   * the use of [[map]] and [[flatMap]] is preferable.
+   */
+  def mapPartition[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("MapPartition function must not be null.")
+    }
+    val partitionMapper = new MapPartitionFunction[T, R] {
+      def mapPartition(in: java.lang.Iterable[T], out: Collector[R]) {
+        fun(in.iterator().asScala, out)
+      }
+    }
+    wrap(new MapPartitionOperator[T, R](set, implicitly[TypeInformation[R]], partitionMapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to each parallel partition of the
+   * DataSet.
+   *
+   * This function is intended for operations that cannot transform individual elements and
+   * requires no grouping of elements. To transform individual elements,
+   * the use of [[map]] and [[flatMap]] is preferable.
+   */
+  def mapPartition[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T]) => TraversableOnce[R]): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("MapPartition function must not be null.")
+    }
+    val partitionMapper = new MapPartitionFunction[T, R] {
+      def mapPartition(in: java.lang.Iterable[T], out: Collector[R]) {
+        fun(in.iterator().asScala) foreach out.collect
+      }
+    }
+    wrap(new MapPartitionOperator[T, R](set, implicitly[TypeInformation[R]], partitionMapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to every element and flattening
+   * the results.
+   */
+  def flatMap[R: TypeInformation: ClassTag](flatMapper: FlatMapFunction[T, R]): DataSet[R] = {
+    if (flatMapper == null) {
+      throw new NullPointerException("FlatMap function must not be null.")
+    }
+    wrap(new FlatMapOperator[T, R](set, implicitly[TypeInformation[R]], flatMapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to every element and flattening
+   * the results.
+   */
+  def flatMap[R: TypeInformation: ClassTag](fun: (T, Collector[R]) => Unit): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("FlatMap function must not be null.")
+    }
+    val flatMapper = new FlatMapFunction[T, R] {
+      def flatMap(in: T, out: Collector[R]) { fun(in, out) }
+    }
+    wrap(new FlatMapOperator[T, R](set, implicitly[TypeInformation[R]], flatMapper))
+  }
+
+  /**
+   * Creates a new DataSet by applying the given function to every element and flattening
+   * the results.
+   */
+  def flatMap[R: TypeInformation: ClassTag](fun: (T) => TraversableOnce[R]): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("FlatMap function must not be null.")
+    }
+    val flatMapper = new FlatMapFunction[T, R] {
+      def flatMap(in: T, out: Collector[R]) { fun(in) foreach out.collect }
+    }
+    wrap(new FlatMapOperator[T, R](set, implicitly[TypeInformation[R]], flatMapper))
+  }
+
+  /**
+   * Creates a new DataSet that contains only the elements satisfying the given filter predicate.
+   */
+  def filter(filter: FilterFunction[T]): DataSet[T] = {
+    if (filter == null) {
+      throw new NullPointerException("Filter function must not be null.")
+    }
+    wrap(new FilterOperator[T](set, filter))
+  }
+
+  /**
+   * Creates a new DataSet that contains only the elements satisfying the given filter predicate.
+   */
+  def filter(fun: (T) => Boolean): DataSet[T] = {
+    if (fun == null) {
+      throw new NullPointerException("Filter function must not be null.")
+    }
+    val filter = new FilterFunction[T] {
+      def filter(in: T) = fun(in)
+    }
+    wrap(new FilterOperator[T](set, filter))
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Non-grouped aggregations
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new [[DataSet]] by aggregating the specified tuple field using the given aggregation
+   * function. Since this is not a keyed DataSet the aggregation will be performed on the whole
+   * collection of elements.
+   *
+   * This only works on Tuple DataSets.
+   */
+  def aggregate(agg: Aggregations, field: Int): DataSet[T] = set match {
+    case aggregation: ScalaAggregateOperator[T] =>
+      aggregation.and(agg, field)
+      wrap(aggregation)
+
+    case _ => wrap(new ScalaAggregateOperator[T](set, agg, field))
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `SUM`
+   */
+  def sum(field: Int) = {
+    aggregate(Aggregations.SUM, field)
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MAX`
+   */
+  def max(field: Int) = {
+    aggregate(Aggregations.MAX, field)
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MIN`
+   */
+  def min(field: Int) = {
+    aggregate(Aggregations.MIN, field)
+  }
+
+  /**
+   * Creates a new [[DataSet]] by merging the elements of this DataSet using an associative reduce
+   * function.
+   */
+  def reduce(reducer: ReduceFunction[T]): DataSet[T] = {
+    if (reducer == null) {
+      throw new NullPointerException("Reduce function must not be null.")
+    }
+    wrap(new ReduceOperator[T](set, reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by merging the elements of this DataSet using an associative reduce
+   * function.
+   */
+  def reduce(fun: (T, T) => T): DataSet[T] = {
+    if (fun == null) {
+      throw new NullPointerException("Reduce function must not be null.")
+    }
+    val reducer = new ReduceFunction[T] {
+      def reduce(v1: T, v2: T) = { fun(v1, v2) }
+    }
+    wrap(new ReduceOperator[T](set, reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by passing all elements in this DataSet to the group reduce function.
+   * The function can output zero or more elements using the [[Collector]]. The concatenation of the
+   * emitted values will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](reducer: GroupReduceFunction[T, R]): DataSet[R] = {
+    if (reducer == null) {
+      throw new NullPointerException("GroupReduce function must not be null.")
+    }
+    wrap(new GroupReduceOperator[T, R](set, implicitly[TypeInformation[R]], reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by passing all elements in this DataSet to the group reduce function.
+   * The function can output zero or more elements using the [[Collector]]. The concatenation of the
+   * emitted values will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("GroupReduce function must not be null.")
+    }
+    val reducer = new GroupReduceFunction[T, R] {
+      def reduce(in: java.lang.Iterable[T], out: Collector[R]) { fun(in.iterator().asScala, out) }
+    }
+    wrap(new GroupReduceOperator[T, R](set, implicitly[TypeInformation[R]], reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by passing all elements in this DataSet to the group reduce function.
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](fun: (TraversableOnce[T]) => R): DataSet[R] = {
+    if (fun == null) {
+      throw new NullPointerException("GroupReduce function must not be null.")
+    }
+    val reducer = new GroupReduceFunction[T, R] {
+      def reduce(in: java.lang.Iterable[T], out: Collector[R]) {
+        out.collect(fun(in.iterator().asScala))
+      }
+    }
+    wrap(new GroupReduceOperator[T, R](set, implicitly[TypeInformation[R]], reducer))
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  distinct
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new DataSet containing the distinct elements of this DataSet. The decision whether
+   * two elements are distinct or not is made using the return value of the given function.
+   */
+  def distinct[K: TypeInformation](fun: (T) => K): DataSet[T] = {
+    val keyExtractor = new KeySelector[T, K] {
+      def getKey(in: T) = fun(in)
+    }
+    wrap(new DistinctOperator[T](
+      set,
+      new Keys.SelectorFunctionKeys[T, K](
+        keyExtractor, set.getType, implicitly[TypeInformation[K]])))
+  }
+
+  /**
+   * Creates a new DataSet containing the distinct elements of this DataSet. The decision whether
+   * two elements are distinct or not is made based on only the specified tuple fields.
+   *
+   * This only works if this DataSet contains Tuples.
+   */
+  def distinct(fields: Int*): DataSet[T] = {
+    wrap(new DistinctOperator[T](
+      set,
+      new Keys.FieldPositionKeys[T](fields.toArray, set.getType, true)))
+  }
+
+  /**
+   * Creates a new DataSet containing the distinct elements of this DataSet. The decision whether
+   * two elements are distinct or not is made based on all tuple fields.
+   *
+   * This only works if this DataSet contains Tuples.
+   */
+  def distinct: DataSet[T] = {
+    wrap(new DistinctOperator[T](set, null))
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Keyed DataSet
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a [[GroupedDataSet]] which provides operations on groups of elements. Elements are
+   * grouped based on the value returned by the given function.
+   *
+   * This will not create a new DataSet, it will just attach the key function which will be used
+   * for grouping when executing a grouped operation.
+   */
+  def groupBy[K: TypeInformation](fun: (T) => K): GroupedDataSet[T] = {
+    val keyType = implicitly[TypeInformation[K]]
+    val keyExtractor = new KeySelector[T, K] {
+      def getKey(in: T) = fun(in)
+    }
+    new GroupedDataSetImpl[T](set,
+      new Keys.SelectorFunctionKeys[T, K](keyExtractor, set.getType, keyType))
+  }
+
+  /**
+   * Creates a [[GroupedDataSet]] which provides operations on groups of elements. Elements are
+   * grouped based on the given tuple fields.
+   *
+   * This will not create a new DataSet, it will just attach the tuple field positions which will be
+   * used for grouping when executing a grouped operation.
+   *
+   * This only works on Tuple DataSets.
+   */
+  def groupBy(fields: Int*): GroupedDataSet[T] = {
+    new GroupedDataSetImpl[T](
+      set,
+      new Keys.FieldPositionKeys[T](fields.toArray, set.getType,false))
+  }
+
+  //	public UnsortedGrouping<T> groupBy(String... fields) {
+  //		new UnsortedGrouping<T>(this, new Keys.ExpressionKeys<T>(fields, getType()));
+  //	}
+
+  // --------------------------------------------------------------------------------------------
+  //  Joining
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new DataSet by joining `this` DataSet with the `other` DataSet. To specify the join
+   * keys the `where` and `isEqualTo` methods must be used. For example:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val joined = left.join(right).where(0).isEqualTo(1)
+   * }}}
+   *
+   * The default join result is a DataSet with 2-Tuples of the joined values. In the above example
+   * that would be `((String, Int, Int), (Int, String, Int))`. A custom join function can be used
+   * if more control over the result is required. This can either be given as a lambda or a
+   * custom [[JoinFunction]]. For example:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val joined = left.join(right).where(0).isEqualTo(1) { (l, r) =>
+   *     if (l._2 > 4) {
+   *       Some((l._2, r._3))
+   *     } else {
+   *       None
+   *     }
+   *   }
+   * }}}
+   * This can be used to implement a filter directly in the join or to output more than one values:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val joined = left.join(right).where(0).isEqualTo(1) {
+   *     (l, r, out: Collector[(String, Int)]) =>
+   *       if (l._2 > 4) {
+   *         out.collect((l._1, r._3))
+   *         out.collect((l._1, r._1))
+   *       } else {
+   *         None
+   *       }
+   *     }
+   * }}}
+   */
+  def join[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperationImpl(this.set, other.set, JoinHint.OPTIMIZER_CHOOSES)
+
+  /**
+   * Special [[join]] operation for explicitly telling the system that the right side is assumed
+   * to be a lot smaller than the left side of the join.
+   */
+  def joinWithTiny[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperationImpl(this.set, other.set, JoinHint.BROADCAST_HASH_SECOND)
+
+  /**
+   * Special [[join]] operation for explicitly telling the system that the left side is assumed
+   * to be a lot smaller than the right side of the join.
+   */
+  def joinWithHuge[O](other: DataSet[O]): UnfinishedJoinOperation[T, O] =
+    new UnfinishedJoinOperationImpl(this.set, other.set, JoinHint.BROADCAST_HASH_FIRST)
+
+  // --------------------------------------------------------------------------------------------
+  //  Co-Group
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * For each key in `this` DataSet and the `other` DataSet, create a tuple containing a list
+   * of elements for that key from both DataSets. To specify the join keys the `where` and
+   * `isEqualTo` methods must be used. For example:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val coGrouped = left.coGroup(right).where(0).isEqualTo(1)
+   * }}}
+   *
+   * A custom coGroup function can be used
+   * if more control over the result is required. This can either be given as a lambda or a
+   * custom [[CoGroupFunction]]. For example:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val coGrouped = left.coGroup(right).where(0).isEqualTo(1) { (l, r) =>
+   *     // l and r are of type TraversableOnce
+   *     Some((l.min, r.max))
+   *   }
+   * }}}
+   * This can be used to implement a filter directly in the coGroup or to output more than one
+   * values:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val coGrouped = left.coGroup(right).where(0).isEqualTo(1) {
+   *     (l, r, out: Collector[(String, Int)]) =>
+   *       out.collect((l.min, r.max))
+   *       out.collect(l.max, r.min))
+   *     }
+   * }}}
+   */
+  def coGroup[O: ClassTag](other: DataSet[O]): UnfinishedCoGroupOperation[T, O] =
+    new UnfinishedCoGroupOperationImpl(this.set, other.set)
+
+  // --------------------------------------------------------------------------------------------
+  //  Cross
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new DataSet by forming the cartesian product of `this` DataSet and the `other`
+   * DataSet.
+   *
+   * The default cross result is a DataSet with 2-Tuples of the combined values. A custom cross
+   * function can be used if more control over the result is required. This can either be given as
+   * a lambda or a custom [[CrossFunction]]. For example:
+   * {{{
+   *   val left: DataSet[(String, Int, Int)] = ...
+   *   val right: DataSet[(Int, String, Int)] = ...
+   *   val product = left.cross(right) { (l, r) => (l._2, r._3) }
+   *   }
+   * }}}
+   */
+  def cross[O](other: DataSet[O]): CrossDataSet[T, O] =
+    CrossDataSetImpl.createCrossOperator(this.set, other.set)
+
+  /**
+   * Special [[cross]] operation for explicitly telling the system that the right side is assumed
+   * to be a lot smaller than the left side of the cartesian product.
+   */
+  def crossWithTiny[O](other: DataSet[O]): CrossDataSet[T, O] =
+    CrossDataSetImpl.createCrossOperator(this.set, other.set)
+
+  /**
+   * Special [[cross]] operation for explicitly telling the system that the left side is assumed
+   * to be a lot smaller than the right side of the cartesian product.
+   */
+  def crossWithHuge[O](other: DataSet[O]): CrossDataSet[T, O] =
+    CrossDataSetImpl.createCrossOperator(this.set, other.set)
+
+  // --------------------------------------------------------------------------------------------
+  //  Iterations
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new DataSet by performing bulk iterations using the given step function. The
+   * iterations terminate when `maxIterations` iterations have been performed.
+   *
+   * For example:
+   * {{{
+   *   val input: DataSet[(String, Int)] = ...
+   *   val iterated = input.iterate(5) { previous =>
+   *     val next = previous.map { x => (x._1, x._2 + 1) }
+   *     next
+   *   }
+   * }}}
+   *
+   * This example will simply increase the second field of the tuple by 5.
+   */
+  def iterate(maxIterations: Int)(stepFunction: (DataSet[T]) => DataSet[T]): DataSet[T] = {
+    val iterativeSet =
+      new IterativeDataSet[T](set.getExecutionEnvironment, set.getType, set,maxIterations)
+
+    val resultSet = stepFunction(wrap(iterativeSet))
+    val result = iterativeSet.closeWith(resultSet.set)
+    wrap(result)
+  }
+
+  /**
+   * Creates a new DataSet by performing bulk iterations using the given step function. The first
+   * DataSet the step function returns is the input for the next iteration, the second DataSet is
+   * the termination criterion. The iterations terminate when either the termination criterion
+   * DataSet contains no elements or when `maxIterations` iterations have been performed.
+   *
+   *  For example:
+   * {{{
+   *   val input: DataSet[(String, Int)] = ...
+   *   val iterated = input.iterateWithTermination(5) { previous =>
+   *     val next = previous.map { x => (x._1, x._2 + 1) }
+   *     val term = next.filter { _._2 <  3 }
+   *     (next, term)
+   *   }
+   * }}}
+   *
+   * This example will simply increase the second field of the Tuples until they are no longer
+   * smaller than 3.
+   */
+  def iterateWithTermination(maxIterations: Int)(
+    stepFunction: (DataSet[T]) => (DataSet[T], DataSet[_])): DataSet[T] = {
+    val iterativeSet =
+      new IterativeDataSet[T](set.getExecutionEnvironment, set.getType, set,maxIterations)
+
+    val (resultSet, terminationCriterion) = stepFunction(wrap(iterativeSet))
+    val result = iterativeSet.closeWith(resultSet.set, terminationCriterion.set)
+    wrap(result)
+  }
+
+  /**
+   * Creates a new DataSet by performing delta (or workset) iterations using the given step
+   * function. At the beginning `this` DataSet is the solution set and `workset` is  the Workset.
+   * The iteration step function gets the current solution set and workset and must output the
+   * delta for the solution set and the workset for the next iteration.
+   *
+   * Note: The syntax of delta iterations are very likely going to change soon.
+   */
+  def iterateDelta[R: ClassTag](workset: DataSet[R], maxIterations: Int, keyFields: Array[Int])(
+      stepFunction: (DataSet[T], DataSet[R]) => (DataSet[T], DataSet[R])) = {
+    val key = new FieldPositionKeys[T](keyFields, set.getType, false)
+    val iterativeSet = new DeltaIteration[T, R](
+      set.getExecutionEnvironment, set.getType, set, workset.set, key, maxIterations)
+    val (newSolution, newWorkset) = stepFunction(
+      wrap(iterativeSet.getSolutionSet),
+      wrap(iterativeSet.getWorkset))
+    val result = iterativeSet.closeWith(newSolution.set, newWorkset.set)
+    wrap(result)
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Union
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Creates a new DataSet containing the elements from both `this` DataSet and the `other`
+   * DataSet.
+   */
+  def union(other: DataSet[T]): DataSet[T] = wrap(new UnionOperator[T](set, other.set))
+
+  // --------------------------------------------------------------------------------------------
+  //  Result writing
+  // --------------------------------------------------------------------------------------------
+
+  /**
+   * Writes `this` DataSet to the specified location. This uses [[AnyRef.toString]] on
+   * each element.
+   */
+  def writeAsText(
+      filePath: String,
+      writeMode: FileSystem.WriteMode = WriteMode.NO_OVERWRITE): DataSink[T] = {
+    val tof: TextOutputFormat[T] = new TextOutputFormat[T](new Path(filePath))
+    tof.setWriteMode(writeMode)
+    output(tof)
+  }
+
+  /**
+   * Writes `this` DataSet to the specified location as a CSV file.
+   *
+   * This only works on Tuple DataSets. For individual tuple fields [[AnyRef.toString]] is used.
+   */
+  def writeAsCsv(
+      filePath: String,
+      rowDelimiter: String = ScalaCsvOutputFormat.DEFAULT_LINE_DELIMITER,
+      fieldDelimiter: String = ScalaCsvOutputFormat.DEFAULT_FIELD_DELIMITER,
+      writeMode: FileSystem.WriteMode = WriteMode.NO_OVERWRITE): DataSink[T] = {
+    Validate.isTrue(set.getType.isTupleType, "CSV output can only be used with Tuple DataSets.")
+    val of = new ScalaCsvOutputFormat[Product](new Path(filePath), rowDelimiter, fieldDelimiter)
+    of.setWriteMode(writeMode)
+    output(of.asInstanceOf[OutputFormat[T]])
+  }
+
+  /**
+   * Writes `this` DataSet to the specified location using a custom
+   * [[org.apache.flink.api.common.io.FileOutputFormat]].
+   */
+  def write(
+      outputFormat: FileOutputFormat[T],
+      filePath: String,
+      writeMode: FileSystem.WriteMode = WriteMode.NO_OVERWRITE): DataSink[T] = {
+    Validate.notNull(filePath, "File path must not be null.")
+    Validate.notNull(outputFormat, "Output format must not be null.")
+    outputFormat.setOutputFilePath(new Path(filePath))
+    outputFormat.setWriteMode(writeMode)
+    output(outputFormat)
+  }
+
+  /**
+   * Emits `this` DataSet using a custom [[org.apache.flink.api.common.io.OutputFormat]].
+   */
+  def output(outputFormat: OutputFormat[T]): DataSink[T] = {
+    set.output(outputFormat)
+  }
+
+  /**
+   * Writes a DataSet to the standard output stream (stdout). This uses [[AnyRef.toString]] on
+   * each element.
+   */
+  def print(): DataSink[T] = {
+    output(new PrintingOutputFormat[T](false))
+  }
+
+  /**
+   * Writes a DataSet to the standard error stream (stderr).This uses [[AnyRef.toString]] on
+   * each element.
+   */
+  def printToErr(): DataSink[T] = {
+    output(new PrintingOutputFormat[T](true))
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSink.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSink.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSink.scala
deleted file mode 100644
index 35d0dd0..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSink.scala
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import java.net.URI
-
-import org.apache.flink.api.scala.analysis._
-
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.api.common.io.FileOutputFormat
-import org.apache.flink.types.Record
-import org.apache.flink.api.common.operators.base.GenericDataSinkBase
-import org.apache.flink.api.java.record.operators.FileDataSink
-import org.apache.flink.api.common.io.OutputFormat
-
-
-object DataSinkOperator {
-  val DEFAULT_DATASINKOPERATOR_NAME = "<Unnamed Scala Data Sink>"
-
-  def write[In](input: DataSet[In], url: String, format: ScalaOutputFormat[In],
-                name: String = DEFAULT_DATASINKOPERATOR_NAME): ScalaSink[In]
-  = {
-    val uri = getUri(url)
-
-    val ret = uri.getScheme match {
-      case "file" | "hdfs" => new FileDataSink(format.asInstanceOf[FileOutputFormat[Record]], uri.toString,
-        input.contract, name) with ScalaOutputOperator[In] {
-
-        def getUDF = format.getUDF
-        override def persistConfiguration() = format.persistConfiguration(this.getParameters())
-      }
-    }
-    new ScalaSink(ret)
-  }
-
-  private def getUri(url: String) = {
-    val uri = new URI(url)
-    if (uri.getScheme == null)
-      new URI("file://" + url)
-    else
-      uri
-  }
-}
-
-class ScalaSink[In] private[scala] (private[scala] val sink: GenericDataSinkBase[Record])
-
-trait ScalaOutputFormat[In] { this: OutputFormat[_] =>
-  def getUDF: UDF1[In, Nothing]
-  def persistConfiguration(config: Configuration) = {}
-  def configure(config: Configuration)
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSource.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSource.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSource.scala
deleted file mode 100644
index e7990a2..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSource.scala
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import java.net.URI
-import collection.JavaConversions._
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.functions._
-import org.apache.flink.api.scala.analysis.UDF0
-import org.apache.flink.types._
-import org.apache.flink.types.parser._
-import org.apache.flink.api.java.record.operators.{CollectionDataSource => JavaCollectionDataSource, FileDataSource}
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.api.common.io.FileInputFormat
-import org.apache.flink.api.java.record.operators.{CollectionDataSource => JavaCollectionDataSource}
-import org.apache.flink.api.common.io.InputFormat
-import org.apache.flink.api.scala.operators.TextInputFormat
-
-
-
-object DataSource {
-
-  def apply[Out](url: String, format: ScalaInputFormat[Out]): DataSet[Out] with OutputHintable[Out] = {
-    val uri = getUri(url)
-    
-    val ret = uri.getScheme match {
-
-      case "file" | "hdfs" => new FileDataSource(format.asInstanceOf[FileInputFormat[Record]], uri.toString)
-          with ScalaOperator[Out, Record] {
-
-        override def getUDF = format.getUDF
-
-        override def persistConfiguration() = format.persistConfiguration(this.getParameters)
-      }
-
-//      case "ext" => new GenericDataSource[GenericInputFormat[_]](format.asInstanceOf[GenericInputFormat[_]], uri.toString)
-//          with ScalaOperator[Out] {
-//
-//        override def getUDF = format.getUDF
-//        override def persistConfiguration() = format.persistConfiguration(this.getParameters())
-//      }
-    }
-    
-    new DataSet[Out](ret) with OutputHintable[Out] {}
-  }
-
-  private def getUri(url: String) = {
-    val uri = new URI(url)
-    if (uri.getScheme == null)
-      new URI("file://" + url)
-    else
-      uri
-  }
-}
-
-object CollectionDataSource {
-  /*
-  constructor for collection input
-   */
-  def apply[Out: UDT](data: Iterable[Out]):DataSet[Out] with OutputHintable[Out] = {
-    /*
-    reuse the java implementation of collection data by adding scala operator
-    */
-    val js:java.util.Collection[Out] = data
-    val ret = new JavaCollectionDataSource(js)
-    	with ScalaOperator[Out, Record]{
-       
-       val udf = new UDF0(implicitly[UDT[Out]])
-       override def getUDF = udf
-
-    }
-    
-    new DataSet[Out](ret) with OutputHintable[Out] {}
-  }
-  
-  /*
-  constructor for serializable iterator input
-   */
-  def apply[Out: UDT](data: Iterator[Out] with Serializable) = {
-
-    /*
-    reuse the java implementation of collection data by adding scala operator
-     */
-    val ret = new JavaCollectionDataSource(data)
-    	with ScalaOperator[Out, Record]{
-       
-       val udf = new UDF0(implicitly[UDT[Out]])
-       override def getUDF = udf
-
-    }
-    
-    new DataSet[Out](ret) with OutputHintable[Out] {}
-  }
-}
-
-
-
-trait ScalaInputFormat[Out] { this: InputFormat[_, _] =>
-  def getUDF: UDF0[Out]
-  def persistConfiguration(config: Configuration) = {}
-  def configure(config: Configuration)
-}
-
-
-object TextFile {
-  def apply(url: String): DataSet[String] with OutputHintable[String] = DataSource(url, TextInputFormat())
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
new file mode 100644
index 0000000..6f44e68
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -0,0 +1,413 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala
+
+import java.util.UUID
+
+import org.apache.commons.lang3.Validate
+import org.apache.flink.api.common.JobExecutionResult
+import org.apache.flink.api.java.io._
+import org.apache.flink.api.java.typeutils.{TupleTypeInfoBase, BasicTypeInfo}
+import org.apache.flink.api.scala.operators.ScalaCsvInputFormat
+import org.apache.flink.core.fs.Path
+
+import org.apache.flink.api.java.{ExecutionEnvironment => JavaEnv}
+import org.apache.flink.api.common.io.{InputFormat, FileInputFormat}
+
+import org.apache.flink.api.java.operators.DataSource
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.{NumberSequenceIterator, SplittableIterator}
+
+import scala.collection.JavaConverters._
+
+import scala.reflect.ClassTag
+
+/**
+ * The ExecutionEnviroment is the context in which a program is executed. A local environment will
+ * cause execution in the current JVM, a remote environment will cause execution on a remote
+ * cluster installation.
+ *
+ * The environment provides methods to control the job execution (such as setting the parallelism)
+ * and to interact with the outside world (data access).
+ *
+ * To get an execution environment use the methods on the companion object:
+ *
+ *  - [[ExecutionEnvironment.getExecutionEnvironment]]
+ *  - [[ExecutionEnvironment.createLocalEnvironment]]
+ *  - [[ExecutionEnvironment.createRemoteEnvironment]]
+ *
+ *  Use [[ExecutionEnvironment.getExecutionEnvironment]] to get the correct environment depending
+ *  on where the program is executed. If it is run inside an IDE a loca environment will be
+ *  created. If the program is submitted to a cluster a remote execution environment will
+ *  be created.
+ */
+class ExecutionEnvironment(javaEnv: JavaEnv) {
+
+  /**
+   * Sets the degree of parallelism (DOP) for operations executed through this environment.
+   * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with
+   * x parallel instances. This value can be overridden by specific operations using
+   * [[DataSet.setParallelism]].
+   */
+  def setDegreeOfParallelism(degreeOfParallelism: Int): Unit = {
+    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+  }
+
+  /**
+   * Returns the default degree of parallelism for this execution environment. Note that this
+   * value can be overridden by individual operations using [[DataSet.setParallelism]]
+   */
+  def getDegreeOfParallelism = javaEnv.getDegreeOfParallelism
+
+  /**
+   * Gets the UUID by which this environment is identified. The UUID sets the execution context
+   * in the cluster or local environment.
+   */
+  def getId: UUID = {
+    javaEnv.getId
+  }
+
+  /**
+   * Gets the UUID by which this environment is identified, as a string.
+   */
+  def getIdString: String = {
+    javaEnv.getIdString
+  }
+
+  /**
+   * Creates a DataSet of Strings produced by reading the given file line wise.
+   *
+   * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or
+   *                 "hdfs://host:port/file/path").
+   * @param charsetName The name of the character set used to read the file. Default is UTF-0
+   */
+  def readTextFile(filePath: String, charsetName: String = "UTF-8"): DataSet[String] = {
+    Validate.notNull(filePath, "The file path may not be null.")
+    val format = new TextInputFormat(new Path(filePath))
+    format.setCharsetName(charsetName)
+    val source = new DataSource[String](javaEnv, format, BasicTypeInfo.STRING_TYPE_INFO)
+    wrap(source)
+  }
+
+  /**
+   * Creates a DataSet by reading the given CSV file. The type parameter must be used to specify
+   * a Tuple type that has the same number of fields as there are fields in the CSV file. If the
+   * number of fields in the CSV file is not the same, the `includedFields` parameter can be used
+   * to only read specific fields.
+   *
+   * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or
+   *                 "hdfs://host:port/file/path").   * @param lineDelimiter
+   * @param lineDelimiter The string that separates lines, defaults to newline.
+   * @param fieldDelimiter The char that separates individual fields, defaults to ','.
+   * @param ignoreFirstLine Whether the first line in the file should be ignored.
+   * @param lenient Whether the parser should silently ignore malformed lines.
+   * @param includedFields The fields in the file that should be read. Per default all fields
+   *                       are read.
+   */
+  def readCsvFile[T <: Product : ClassTag : TypeInformation](
+      filePath: String,
+      lineDelimiter: String = "\n",
+      fieldDelimiter: Char = ',',
+      ignoreFirstLine: Boolean = false,
+      lenient: Boolean = false,
+      includedFields: Array[Int] = null): DataSet[T] = {
+
+    val typeInfo = implicitly[TypeInformation[T]].asInstanceOf[TupleTypeInfoBase[T]]
+
+    val inputFormat = new ScalaCsvInputFormat[T](new Path(filePath), typeInfo)
+    inputFormat.setDelimiter(lineDelimiter)
+    inputFormat.setFieldDelimiter(fieldDelimiter)
+    inputFormat.setSkipFirstLineAsHeader(ignoreFirstLine)
+    inputFormat.setLenient(lenient)
+
+    val classes: Array[Class[_]] = new Array[Class[_]](typeInfo.getArity)
+    for (i <- 0 until typeInfo.getArity) {
+      classes(i) = typeInfo.getTypeAt(i).getTypeClass
+    }
+    if (includedFields != null) {
+      Validate.isTrue(typeInfo.getArity == includedFields.length, "Number of tuple fields and" +
+        " included fields must match.")
+      inputFormat.setFields(includedFields, classes)
+    } else {
+      inputFormat.setFieldTypes(classes)
+    }
+
+    wrap(new DataSource[T](javaEnv, inputFormat, typeInfo))
+  }
+
+  /**
+   * Creates a new DataSource by reading the specified file using the custom
+   * [[org.apache.flink.api.common.io.FileInputFormat]].
+   */
+  def readFile[T : ClassTag : TypeInformation](
+      inputFormat: FileInputFormat[T],
+      filePath: String): DataSet[T] = {
+    Validate.notNull(inputFormat, "InputFormat must not be null.")
+    Validate.notNull(filePath, "File path must not be null.")
+    inputFormat.setFilePath(new Path(filePath))
+    createInput(inputFormat, implicitly[TypeInformation[T]])
+  }
+
+  /**
+   * Generic method to create an input DataSet with an
+   * [[org.apache.flink.api.common.io.InputFormat]].
+   */
+  def createInput[T : ClassTag : TypeInformation](inputFormat: InputFormat[T, _]): DataSet[T] = {
+    if (inputFormat == null) {
+      throw new IllegalArgumentException("InputFormat must not be null.")
+    }
+    createInput(inputFormat, implicitly[TypeInformation[T]])
+  }
+
+  /**
+   * Generic method to create an input DataSet with an
+   * [[org.apache.flink.api.common.io.InputFormat]].
+   */
+  private def createInput[T: ClassTag](
+      inputFormat: InputFormat[T, _],
+      producedType: TypeInformation[T]): DataSet[T] = {
+    if (inputFormat == null) {
+      throw new IllegalArgumentException("InputFormat must not be null.")
+    }
+    Validate.notNull(producedType, "Produced type must not be null")
+    wrap(new DataSource[T](javaEnv, inputFormat, producedType))
+  }
+
+  /**
+   * Creates a DataSet from the given non-empty [[Seq]]. The elements need to be serializable
+   * because the framework may move the elements into the cluster if needed.
+   *
+   * Note that this operation will result in a non-parallel data source, i.e. a data source with
+   * a degree of parallelism of one.
+   */
+  def fromCollection[T: ClassTag : TypeInformation](
+      data: Seq[T]): DataSet[T] = {
+    Validate.notNull(data, "Data must not be null.")
+
+    val typeInfo = implicitly[TypeInformation[T]]
+    CollectionInputFormat.checkCollection(data.asJavaCollection, typeInfo.getTypeClass)
+    val dataSource = new DataSource[T](
+      javaEnv,
+      new CollectionInputFormat[T](data.asJavaCollection, typeInfo.createSerializer),
+      typeInfo)
+    wrap(dataSource)
+  }
+
+  /**
+   * Creates a DataSet from the given [[Iterator]]. The iterator must be serializable because the
+   * framework might move into the cluster if needed.
+   *
+   * Note that this operation will result in a non-parallel data source, i.e. a data source with
+   * a degree of parallelism of one.
+   */
+  def fromCollection[T: ClassTag : TypeInformation] (
+    data: Iterator[T]): DataSet[T] = {
+    Validate.notNull(data, "Data must not be null.")
+
+    val typeInfo = implicitly[TypeInformation[T]]
+    val dataSource = new DataSource[T](
+      javaEnv,
+      new IteratorInputFormat[T](data.asJava),
+      typeInfo)
+    wrap(dataSource)
+  }
+
+  /**
+   * Creates a new data set that contains the given elements. The elements must all be of the
+   * same type and must be serializable.
+   *
+   * * Note that this operation will result in a non-parallel data source, i.e. a data source with
+   * a degree of parallelism of one.
+   */
+  def fromElements[T: ClassTag : TypeInformation](data: T*): DataSet[T] = {
+    Validate.notNull(data, "Data must not be null.")
+    val typeInfo = implicitly[TypeInformation[T]]
+    fromCollection(data)(implicitly[ClassTag[T]], typeInfo)
+  }
+
+  /**
+   * Creates a new data set that contains elements in the iterator. The iterator is splittable,
+   * allowing the framework to create a parallel data source that returns the elements in the
+   * iterator. The iterator must be serializable because the execution environment may ship the
+   * elements into the cluster.
+   */
+  def fromParallelCollection[T: ClassTag : TypeInformation](
+      iterator: SplittableIterator[T]): DataSet[T] = {
+    val typeInfo = implicitly[TypeInformation[T]]
+    wrap(new DataSource[T](javaEnv, new ParallelIteratorInputFormat[T](iterator), typeInfo))
+  }
+
+  /**
+   * Creates a new data set that contains a sequence of numbers. The data set will be created in
+   * parallel, so there is no guarantee about the oder of the elements.
+   *
+   * @param from The number to start at (inclusive).
+   * @param to The number to stop at (inclusive).
+   */
+  def generateSequence(from: Long, to: Long): DataSet[Long] = {
+    val iterator = new NumberSequenceIterator(from, to)
+    val source = new DataSource(
+      javaEnv,
+      new ParallelIteratorInputFormat[java.lang.Long](iterator),
+      BasicTypeInfo.LONG_TYPE_INFO)
+    wrap(source).asInstanceOf[DataSet[Long]]
+  }
+
+  /**
+   * Registers a file at the distributed cache under the given name. The file will be accessible
+   * from any user-defined function in the (distributed) runtime under a local path. Files
+   * may be local files (as long as all relevant workers have access to it),
+   * or files in a distributed file system.
+   * The runtime will copy the files temporarily to a local cache, if needed.
+   *
+   * The [[org.apache.flink.api.common.functions.RuntimeContext]] can be obtained inside UDFs
+   * via
+   * [[org.apache.flink.api.common.functions.RichFunction#getRuntimeContext]] and provides
+   * access via
+   * [[org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache]]
+   *
+   * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or
+   *                 "hdfs://host:port/and/path")
+   * @param name The name under which the file is registered.
+   * @param executable Flag indicating whether the file should be executable
+   */
+  def registerCachedFile(filePath: String, name: String, executable: Boolean = false): Unit = {
+    javaEnv.registerCachedFile(filePath, name, executable)
+  }
+
+  /**
+   * Triggers the program execution. The environment will execute all parts of the program that have
+   * resulted in a "sink" operation. Sink operations are for example printing results
+   * [[DataSet.print]], writing results (e.g. [[DataSet.writeAsText]], [[DataSet.write]], or other
+   * generic data sinks created with [[DataSet.output]].
+   *
+   * The program execution will be logged and displayed with a generated default name.
+   *
+   * @return The result of the job execution, containing elapsed time and accumulators.
+   */
+  def execute(): JobExecutionResult = {
+    javaEnv.execute()
+  }
+
+  /**
+   * Triggers the program execution. The environment will execute all parts of the program that have
+   * resulted in a "sink" operation. Sink operations are for example printing results
+   * [[DataSet.print]], writing results (e.g. [[DataSet.writeAsText]], [[DataSet.write]], or other
+   * generic data sinks created with [[DataSet.output]].
+   *
+   * The program execution will be logged and displayed with the given name.
+   *
+   * @return The result of the job execution, containing elapsed time and accumulators.
+   */
+  def execute(jobName: String): JobExecutionResult = {
+    javaEnv.execute(jobName)
+  }
+
+  /**
+   * Creates the plan with which the system will execute the program, and returns it as  a String
+   * using a JSON representation of the execution data flow graph.
+   */
+  def getExecutionPlan() = {
+    javaEnv.getExecutionPlan
+  }
+
+  /**
+   * Creates the program's [[org.apache.flink.api.common.Plan]].
+   * The plan is a description of all data sources, data sinks,
+   * and operations and how they interact, as an isolated unit that can be executed with a
+   * [[org.apache.flink.api.common.PlanExecutor]]. Obtaining a plan and starting it with an
+   * executor is an alternative way to run a program and is only possible if the program only
+   * consists of distributed operations.
+   */
+  def createProgramPlan(jobName: String = "") = {
+    if (jobName.isEmpty) {
+      javaEnv.createProgramPlan()
+    } else
+      javaEnv.createProgramPlan(jobName)
+  }
+}
+
+object ExecutionEnvironment {
+
+  /**
+   * Creates an execution environment that represents the context in which the program is
+   * currently executed. If the program is invoked standalone, this method returns a local
+   * execution environment. If the program is invoked from within the command line client
+   * to be submitted to a cluster, this method returns the execution environment of this cluster.
+   */
+  def getExecutionEnvironment: ExecutionEnvironment = {
+    new ExecutionEnvironment(JavaEnv.getExecutionEnvironment)
+  }
+
+  /**
+   * Creates a local execution environment. The local execution environment will run the program in
+   * a multi-threaded fashion in the same JVM as the environment was created in. The default degree
+   * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads).
+   */
+  def createLocalEnvironment(
+      degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()) : ExecutionEnvironment = {
+    val javaEnv = JavaEnv.createLocalEnvironment()
+    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+    new ExecutionEnvironment(javaEnv)
+  }
+
+  /**
+   * Creates a remote execution environment. The remote environment sends (parts of) the program to
+   * a cluster for execution. Note that all file paths used in the program must be accessible from
+   * the cluster. The execution will use the cluster's default degree of parallelism, unless the
+   * parallelism is set explicitly via [[ExecutionEnvironment.setDegreeOfParallelism()]].
+   *
+   * @param host The host name or address of the master (JobManager),
+   *             where the program should be executed.
+   * @param port The port of the master (JobManager), where the program should be executed.
+   * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the
+   *                 program uses
+   *                 user-defined functions, user-defined input formats, or any libraries,
+   *                 those must be
+   *                 provided in the JAR files.
+   */
+  def createRemoteEnvironment(host: String, port: Int, jarFiles: String*): ExecutionEnvironment = {
+    new ExecutionEnvironment(JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*))
+  }
+
+  /**
+   * Creates a remote execution environment. The remote environment sends (parts of) the program
+   * to a cluster for execution. Note that all file paths used in the program must be accessible
+   * from the cluster. The execution will use the specified degree of parallelism.
+   *
+   * @param host The host name or address of the master (JobManager),
+   *             where the program should be executed.
+   * @param port The port of the master (JobManager), where the program should be executed.
+   * @param degreeOfParallelism The degree of parallelism to use during the execution.
+   * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the
+   *                 program uses
+   *                 user-defined functions, user-defined input formats, or any libraries,
+   *                 those must be
+   *                 provided in the JAR files.
+   */
+  def createRemoteEnvironment(
+      host: String,
+      port: Int,
+      degreeOfParallelism: Int,
+      jarFiles: String*): ExecutionEnvironment = {
+    val javaEnv = JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*)
+    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+    new ExecutionEnvironment(javaEnv)
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
new file mode 100644
index 0000000..dfd5cf0
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
@@ -0,0 +1,270 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala
+
+import org.apache.flink.api.common.InvalidProgramException
+import org.apache.flink.api.scala.operators.ScalaAggregateOperator
+
+import scala.collection.JavaConverters._
+
+import org.apache.commons.lang3.Validate
+import org.apache.flink.api.common.functions.{GroupReduceFunction, ReduceFunction}
+import org.apache.flink.api.common.operators.Order
+import org.apache.flink.api.java.aggregation.Aggregations
+import org.apache.flink.api.java.operators._
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.Collector
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+
+import scala.collection.mutable
+import scala.reflect.ClassTag
+
+/**
+ * A [[DataSet]] to which a grouping key was added. Operations work on groups of elements with the
+ * same key (`aggregate`, `reduce`, and `reduceGroup`).
+ *
+ * A secondary sort order can be added with sortGroup, but this is only used when using one
+ * of the group-at-a-time operations, i.e. `reduceGroup`.
+ */
+trait GroupedDataSet[T] {
+
+  /**
+   * Adds a secondary sort key to this [[GroupedDataSet]]. This will only have an effect if you
+   * use one of the group-at-a-time, i.e. `reduceGroup`
+   */
+  def sortGroup(field: Int, order: Order): GroupedDataSet[T]
+
+  /**
+   * Creates a new [[DataSet]] by aggregating the specified tuple field using the given aggregation
+   * function. Since this is a keyed DataSet the aggregation will be performed on groups of
+   * tuples with the same key.
+   *
+   * This only works on Tuple DataSets.
+   */
+  def aggregate(agg: Aggregations, field: Int): DataSet[T]
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `SUM`
+   */
+  def sum(field: Int): DataSet[T]
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MAX`
+   */
+  def max(field: Int): DataSet[T]
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MIN`
+   */
+  def min(field: Int): DataSet[T]
+
+  /**
+   * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
+   * using an associative reduce function.
+   */
+  def reduce(fun: (T, T) => T): DataSet[T]
+
+  /**
+   * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
+   * using an associative reduce function.
+   */
+  def reduce(reducer: ReduceFunction[T]): DataSet[T]
+
+  /**
+   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
+   * of elements to the group reduce function. The function must output one element. The
+   * concatenation of those will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](fun: (TraversableOnce[T]) => R): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
+   * of elements to the group reduce function. The function can output zero or more elements using
+   * the [[Collector]]. The concatenation of the emitted values will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](
+      fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
+   * of elements to the [[GroupReduceFunction]]. The function can output zero or more elements. The
+   * concatenation of the emitted values will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](reducer: GroupReduceFunction[T, R]): DataSet[R]
+}
+
+/**
+ * /**
+ * Private implementation for [[GroupedDataSet]] to keep the implementation details, i.e. the
+ * parameters of the constructor, hidden.
+ */
+ */
+private[flink] class GroupedDataSetImpl[T: ClassTag](
+    private val set: JavaDataSet[T],
+    private val keys: Keys[T])
+  extends GroupedDataSet[T] {
+
+  // These are for optional secondary sort. They are only used
+  // when using a group-at-a-time reduce function.
+  private val groupSortKeyPositions = mutable.MutableList[Int]()
+  private val groupSortOrders = mutable.MutableList[Order]()
+
+  /**
+   * Adds a secondary sort key to this [[GroupedDataSet]]. This will only have an effect if you
+   * use one of the group-at-a-time, i.e. `reduceGroup`
+   */
+  def sortGroup(field: Int, order: Order): GroupedDataSet[T] = {
+    if (!set.getType.isTupleType) {
+      throw new InvalidProgramException("Specifying order keys via field positions is only valid " +
+        "for tuple data types")
+    }
+    if (field >= set.getType.getArity) {
+      throw new IllegalArgumentException("Order key out of tuple bounds.")
+    }
+    groupSortKeyPositions += field
+    groupSortOrders += order
+    this
+  }
+
+  /**
+   * Creates a [[SortedGrouping]] if any secondary sort fields were specified. Otherwise, just
+   * create an [[UnsortedGrouping]].
+   */
+  private def maybeCreateSortedGrouping(): Grouping[T] = {
+    if (groupSortKeyPositions.length > 0) {
+      val grouping = new SortedGrouping[T](set, keys, groupSortKeyPositions(0), groupSortOrders(0))
+      // now manually add the rest of the keys
+      for (i <- 1 until groupSortKeyPositions.length) {
+        grouping.sortGroup(groupSortKeyPositions(i), groupSortOrders(i))
+      }
+      grouping
+    } else {
+      new UnsortedGrouping[T](set, keys)
+    }
+  }
+
+  /** Convenience methods for creating the [[UnsortedGrouping]] */
+  private def createUnsortedGrouping(): Grouping[T] = new UnsortedGrouping[T](set, keys)
+
+  /**
+   * Creates a new [[DataSet]] by aggregating the specified tuple field using the given aggregation
+   * function. Since this is a keyed DataSet the aggregation will be performed on groups of
+   * tuples with the same key.
+   *
+   * This only works on Tuple DataSets.
+   */
+  def aggregate(agg: Aggregations, field: Int): DataSet[T] = set match {
+    case aggregation: ScalaAggregateOperator[T] =>
+      aggregation.and(agg, field)
+      wrap(aggregation)
+
+    case _ => wrap(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, field))
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `SUM`
+   */
+  def sum(field: Int): DataSet[T] = {
+    aggregate(Aggregations.SUM, field)
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MAX`
+   */
+  def max(field: Int): DataSet[T] = {
+    aggregate(Aggregations.MAX, field)
+  }
+
+  /**
+   * Syntactic sugar for [[aggregate]] with `MIN`
+   */
+  def min(field: Int): DataSet[T] = {
+    aggregate(Aggregations.MIN, field)
+  }
+
+  /**
+   * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
+   * using an associative reduce function.
+   */
+  def reduce(fun: (T, T) => T): DataSet[T] = {
+    Validate.notNull(fun, "Reduce function must not be null.")
+    val reducer = new ReduceFunction[T] {
+      def reduce(v1: T, v2: T) = {
+        fun(v1, v2)
+      }
+    }
+    wrap(new ReduceOperator[T](createUnsortedGrouping(), reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
+   * using an associative reduce function.
+   */
+  def reduce(reducer: ReduceFunction[T]): DataSet[T] = {
+    Validate.notNull(reducer, "Reduce function must not be null.")
+    wrap(new ReduceOperator[T](createUnsortedGrouping(), reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
+   * of elements to the group reduce function. The function must output one element. The
+   * concatenation of those will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](
+                                                 fun: (TraversableOnce[T]) => R): DataSet[R] = {
+    Validate.notNull(fun, "Group reduce function must not be null.")
+    val reducer = new GroupReduceFunction[T, R] {
+      def reduce(in: java.lang.Iterable[T], out: Collector[R]) {
+        out.collect(fun(in.iterator().asScala))
+      }
+    }
+    wrap(
+      new GroupReduceOperator[T, R](createUnsortedGrouping(),
+        implicitly[TypeInformation[R]], reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
+   * of elements to the group reduce function. The function can output zero or more elements using
+   * the [[Collector]]. The concatenation of the emitted values will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](
+                                                 fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
+    Validate.notNull(fun, "Group reduce function must not be null.")
+    val reducer = new GroupReduceFunction[T, R] {
+      def reduce(in: java.lang.Iterable[T], out: Collector[R]) {
+        fun(in.iterator().asScala, out)
+      }
+    }
+    wrap(
+      new GroupReduceOperator[T, R](createUnsortedGrouping(),
+        implicitly[TypeInformation[R]], reducer))
+  }
+
+  /**
+   * Creates a new [[DataSet]] by passing for each group (elements with the same key) the list
+   * of elements to the [[GroupReduceFunction]]. The function can output zero or more elements. The
+   * concatenation of the emitted values will form the resulting [[DataSet]].
+   */
+  def reduceGroup[R: TypeInformation: ClassTag](reducer: GroupReduceFunction[T, R]): DataSet[R] = {
+    Validate.notNull(reducer, "GroupReduce function must not be null.")
+    wrap(
+      new GroupReduceOperator[T, R](maybeCreateSortedGrouping(),
+        implicitly[TypeInformation[R]], reducer))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaOperator.scala
deleted file mode 100644
index 1a62d31..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaOperator.scala
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import java.lang.annotation.Annotation
-
-import org.apache.flink.api.scala.analysis.UDF
-import org.apache.flink.api.scala.analysis.UDF0
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.analysis.UDF2
-import org.apache.flink.api.scala.analysis.FieldSelector
-
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.compiler.dag.OptimizerNode
-import org.apache.flink.api.common.operators.AbstractUdfOperator
-import org.apache.flink.api.common.operators.Union
-import org.apache.flink.types.Record
-import org.apache.flink.types.{Nothing => JavaNothing}
-
-trait ScalaOperator[T, UT] {
-  this: Operator[UT] =>
-  def getUDF: UDF[T]
-  def getKeys: Seq[FieldSelector] = Seq()
-  def persistConfiguration(): Unit = {}
-  
-  var persistHints: () => Unit = { () => }
-  
-  def persistConfiguration(optimizerNode: Option[OptimizerNode]): Unit = {
-
-    ScalaOperator.this match {
-      
-      case contract: AbstractUdfOperator[_, _] => {
-        for ((key, inputNum) <- getKeys.zipWithIndex) {
-          
-          val source = key.selectedFields.toSerializerIndexArray
-          val target = optimizerNode map { _.getRemappedKeys(inputNum) } getOrElse {
-            contract.getKeyColumns(inputNum) }
-
-          assert(source.length == target.length, "Attempt to write " + source.length +
-            " key indexes to an array of size " + target.length)
-          System.arraycopy(source, 0, target, 0, source.length)
-        }
-      }
-      
-      case _ if getKeys.size > 0 => throw new UnsupportedOperationException("Attempted to set " +
-        "keys on a contract that doesn't support them")
-      
-      case _ =>
-    }
-
-    persistHints()
-    persistConfiguration()
-  }
-  
-  protected def annotations: Seq[Annotation] = Seq()
-
-  def getUserCodeAnnotation[A <: Annotation](annotationClass: Class[A]): A = {
-    val res = annotations find { _.annotationType().equals(annotationClass) } map {
-      _.asInstanceOf[A] } getOrElse null.asInstanceOf[A]
-//    println("returning ANOOT: " + res + " FOR: " + annotationClass.toString)
-//    res match {
-//      case r : FunctionAnnotation.ConstantFieldsFirst => println("CONSTANT FIELDS FIRST: " +
-//        r.value().mkString(","))
-//      case r : FunctionAnnotation.ConstantFieldsSecond => println("CONSTANT FIELDS SECOND: " +
-//        r.value().mkString(","))
-//      case _ =>
-//    }
-    res
-  }
-}
-
-trait NoOpScalaOperator[In, Out] extends ScalaOperator[Out, Record] { this: Operator[Record] =>
-}
-
-trait HigherOrderScalaOperator[T] extends ScalaOperator[T, Record] { this: Operator[Record] =>
-  override def getUDF: UDF0[T]
-}
-
-trait BulkIterationScalaOperator[T] extends HigherOrderScalaOperator[T] { this: Operator[Record] =>
-}
-
-trait DeltaIterationScalaOperator[T] extends HigherOrderScalaOperator[T] {
-  this: Operator[Record] =>
-  val key: FieldSelector
-}
-
-trait ScalaOutputOperator[In] extends ScalaOperator[Nothing, JavaNothing] {
-  this: Operator[JavaNothing] =>
-  override def getUDF: UDF1[In, Nothing]
-}
-
-trait OneInputScalaOperator[In, Out] extends ScalaOperator[Out, Record] { this: Operator[Record] =>
-  override def getUDF: UDF1[In, Out]
-}
-
-trait TwoInputScalaOperator[In1, In2, Out] extends ScalaOperator[Out, Record] {
-  this: Operator[Record] =>
-  override def getUDF: UDF2[In1, In2, Out]
-}
-
-trait UnionScalaOperator[In] extends TwoInputScalaOperator[In, In, In] {
-  this: Union[Record] =>
-  override def getUDF: UDF2[In, In, In]
-}
-
-trait OneInputKeyedScalaOperator[In, Out] extends OneInputScalaOperator[In, Out] {
-  this: Operator[Record] =>
-  val key: FieldSelector
-  override def getKeys = Seq(key)
-}
-
-trait TwoInputKeyedScalaOperator[LeftIn, RightIn, Out]
-  extends TwoInputScalaOperator[LeftIn, RightIn, Out] {
-
-  this: Operator[Record] =>
-  val leftKey: FieldSelector
-  val rightKey: FieldSelector
-  override def getKeys = Seq(leftKey, rightKey)
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaPlan.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaPlan.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaPlan.scala
deleted file mode 100644
index 2ddd437..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaPlan.scala
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import scala.collection.JavaConversions.asJavaCollection
-
-import java.util.Calendar
-
-import org.apache.flink.api.common.Plan
-import org.apache.flink.compiler.plan.OptimizedPlan
-import org.apache.flink.compiler.postpass.RecordModelPostPass
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.types.Record
-
-import org.apache.flink.api.scala.analysis.GlobalSchemaGenerator
-import org.apache.flink.api.scala.analysis.postPass.GlobalSchemaOptimizer
-
-
-class ScalaPlan(scalaSinks: Seq[ScalaSink[_]], scalaJobName: String = "Flink Scala Job at " + Calendar.getInstance()
-  .getTime()) extends Plan(asJavaCollection(ScalaPlan.setAnnotations(scalaSinks) map { _.sink }), scalaJobName) {
-  val pactSinks = scalaSinks map { _.sink.asInstanceOf[Operator[Record] with ScalaOperator[_, _]] }
-  new GlobalSchemaGenerator().initGlobalSchema(pactSinks)
-  override def getPostPassClassName() = "org.apache.flink.api.scala.ScalaPostPass";
-
-}
-
-object ScalaPlan{
-  def setAnnotations(sinks: Seq[ScalaSink[_]]): Seq[ScalaSink[_]] = {
-    AnnotationUtil.setAnnotations(sinks)
-  }
-}
-
-case class Args(argsMap: Map[String, String], defaultParallelism: Int, schemaHints: Boolean, schemaCompaction: Boolean) {
-  def apply(key: String): String = argsMap.getOrElse(key, key)
-  def apply(key: String, default: => String) = argsMap.getOrElse(key, default)
-}
-
-object Args {
-
-  def parse(args: Seq[String]): Args = {
-
-    var argsMap = Map[String, String]()
-    var defaultParallelism = 1
-    var schemaHints = true
-    var schemaCompaction = true
-
-    val ParamName = "-(.+)".r
-
-    def parse(args: Seq[String]): Unit = args match {
-      case Seq("-subtasks", value, rest @ _*)     => { defaultParallelism = value.toInt; parse(rest) }
-      case Seq("-nohints", rest @ _*)             => { schemaHints = false; parse(rest) }
-      case Seq("-nocompact", rest @ _*)           => { schemaCompaction = false; parse(rest) }
-      case Seq(ParamName(name), value, rest @ _*) => { argsMap = argsMap.updated(name, value); parse(rest) }
-      case Seq()                                  =>
-    }
-
-    parse(args)
-    Args(argsMap, defaultParallelism, schemaHints, schemaCompaction)
-  }
-}
-
-//abstract class ScalaProgram extends Program {
-//  def getScalaPlan(args: Args): ScalaPlan
-//  
-//  override def getPlan(args: String*): Plan = {
-//    val scalaArgs = Args.parse(args.toSeq)
-//    
-//    getScalaPlan(scalaArgs)
-//  }
-//}
-
-
-class ScalaPostPass extends RecordModelPostPass with GlobalSchemaOptimizer {
-  override def postPass(plan: OptimizedPlan): Unit = {
-    optimizeSchema(plan, false)
-    super.postPass(plan)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/Extractors.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/Extractors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/Extractors.scala
deleted file mode 100644
index e08cd0f..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/Extractors.scala
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-import org.apache.flink.compiler.dag._
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.common.operators.Union
-import org.apache.flink.api.common.operators.DualInputOperator
-import org.apache.flink.api.common.operators.SingleInputOperator
-
-import org.apache.flink.api.common.operators.base.{BulkIterationBase => BulkIteration, DeltaIterationBase => DeltaIteration, GenericDataSinkBase, GenericDataSourceBase}
-
-import org.apache.flink.api.java.record.operators.CrossOperator
-import org.apache.flink.api.java.record.operators.CoGroupOperator
-import org.apache.flink.api.java.record.operators.JoinOperator
-import org.apache.flink.api.java.record.operators.MapOperator
-import org.apache.flink.api.java.record.operators.ReduceOperator
-
-import org.apache.flink.types.Record
-
-import org.apache.flink.api.scala._
-
-
-object Extractors {
-
-  object DataSinkNode {
-    def unapply(node: Operator[_]): Option[(UDF1[_, _], Operator[Record])] = node match {
-      case contract: GenericDataSinkBase[_] with ScalaOutputOperator[_] => {
-        Some((contract.getUDF.asInstanceOf[UDF1[_, _]], node.asInstanceOf[GenericDataSinkBase[_]].getInput().asInstanceOf[Operator[Record]]))
-      }
-      case _                               => None
-    }
-  }
-
-  object DataSourceNode {
-    def unapply(node: Operator[_]): Option[(UDF0[_])] = node match {
-      case contract: GenericDataSourceBase[_, _] with ScalaOperator[_, _] => Some(contract.getUDF.asInstanceOf[UDF0[_]])
-      case _                                 => None
-    }
-  }
-
-  object CoGroupNode {
-    def unapply(node: Operator[_]): Option[(UDF2[_, _, _], FieldSelector, FieldSelector, Operator[Record], Operator[Record])] = node match {
-      case contract: CoGroupOperator with TwoInputKeyedScalaOperator[_, _, _] => Some((contract.getUDF, contract.leftKey, contract.rightKey, contract.asInstanceOf[DualInputOperator[_, _, _, _]].getFirstInput().asInstanceOf[Operator[Record]], contract.asInstanceOf[DualInputOperator[_, _, _, _]].getSecondInput().asInstanceOf[Operator[Record]]))
-      case _                                       => None
-    }
-  }
-
-  object CrossNode {
-    def unapply(node: Operator[_]): Option[(UDF2[_, _, _], Operator[Record], Operator[Record])] = node match {
-      case contract: CrossOperator with TwoInputScalaOperator[_, _, _] => Some((contract.getUDF, contract.asInstanceOf[DualInputOperator[_, _, _, _]].getFirstInput().asInstanceOf[Operator[Record]], contract.asInstanceOf[DualInputOperator[_, _, _, _]].getSecondInput().asInstanceOf[Operator[Record]]))
-      case _                                  => None
-    }
-  }
-
-  object JoinNode {
-    def unapply(node: Operator[_]): Option[(UDF2[_, _, _], FieldSelector, FieldSelector, Operator[Record], Operator[Record])] = node match {
-      case contract: JoinOperator with TwoInputKeyedScalaOperator[ _, _, _] => Some((contract.getUDF, contract.leftKey, contract.rightKey, contract.asInstanceOf[DualInputOperator[_, _, _, _]].getFirstInput().asInstanceOf[Operator[Record]], contract.asInstanceOf[DualInputOperator[_, _, _, _]].getSecondInput().asInstanceOf[Operator[Record]]))
-      case _                                    => None
-    }
-  }
-
-  object MapNode {
-    def unapply(node: Operator[_]): Option[(UDF1[_, _], Operator[Record])] = node match {
-      case contract: MapOperator with OneInputScalaOperator[_, _] => Some((contract.getUDF, contract.asInstanceOf[SingleInputOperator[_, _, _]].getInput().asInstanceOf[Operator[Record]]))
-      case _                             => None
-    }
-  }
-  
-  object UnionNode {
-    def unapply(node: Operator[_]): Option[(UDF2[_, _, _], Operator[Record], Operator[Record])] = node match {
-      case contract: Union[_] with UnionScalaOperator[_] => Some((contract.getUDF, contract.asInstanceOf[DualInputOperator[_, _, _, _]].getFirstInput().asInstanceOf[Operator[Record]], contract.asInstanceOf[DualInputOperator[_, _, _, _]].getSecondInput().asInstanceOf[Operator[Record]]))
-      case _                             => None
-    }
-  }
-
-  object ReduceNode {
-    def unapply(node: Operator[_]): Option[(UDF1[_, _], FieldSelector, Operator[Record])] = node match {
-      case contract: ReduceOperator with OneInputKeyedScalaOperator[_, _] => Some((contract.getUDF, contract.key, contract.asInstanceOf[SingleInputOperator[_, _, _]].getInput().asInstanceOf[Operator[Record]]))
-      case contract: ReduceOperator with OneInputScalaOperator[_, _] => Some((contract.getUDF, new FieldSelector(contract.getUDF.inputUDT, Nil), contract.asInstanceOf[SingleInputOperator[_, _, _]].getInput().asInstanceOf[Operator[Record]]))
-      case _                                   => None
-    }
-  }
- object DeltaIterationNode {
-    def unapply(node: Operator[_]): Option[(UDF0[_], FieldSelector, Operator[Record], Operator[Record])] = node match {
-        case contract: DeltaIteration[_, _] with DeltaIterationScalaOperator[_] => Some((contract.getUDF, contract.key, contract.asInstanceOf[DualInputOperator[_, _, _, _]].getFirstInput().asInstanceOf[Operator[Record]], contract.asInstanceOf[DualInputOperator[_, _, _, _]].getSecondInput().asInstanceOf[Operator[Record]]))
-        case _                                  => None
-      }
-  }
-  
-  object BulkIterationNode {
-    def unapply(node: Operator[_]): Option[(UDF0[_], Operator[Record])] = node match {
-      case contract: BulkIteration[_] with BulkIterationScalaOperator[_] => Some((contract.getUDF, contract.asInstanceOf[SingleInputOperator[_, _, _]].getInput().asInstanceOf[Operator[Record]]))
-      case _ => None
-    }
-  } 
-}


[52/60] git commit: Rewrite Java API Guide as Unified Programming Guide

Posted by al...@apache.org.
Rewrite Java API Guide as Unified Programming Guide

This now covers both Java and Scala.


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

Branch: refs/heads/master
Commit: 97d630d5dbf8239818722880b933a582e3223055
Parents: feade05
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 19 11:24:49 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 docs/iterations.md        |    2 +-
 docs/programming_guide.md | 1121 +++++++++++++++++++++++++++++++++++-----
 docs/scala_api_guide.md   | 1042 -------------------------------------
 3 files changed, 1000 insertions(+), 1165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/97d630d5/docs/iterations.md
----------------------------------------------------------------------
diff --git a/docs/iterations.md b/docs/iterations.md
index cb1a1fb..315b9f0 100644
--- a/docs/iterations.md
+++ b/docs/iterations.md
@@ -9,7 +9,7 @@ Iterative algorithms occur in many domains of data analysis, such as *machine le
 
 Flink programs implement iterative algorithms by defining a **step function** and embedding it into a special iteration operator. There are two  variants of this operator: **Iterate** and **Delta Iterate**. Both operators repeatedly invoke the step function on the current iteration state until a certain termination condition is reached.
 
-Here, we provide background on both operator variants and outline their usage. The [programming guides](java_api_guide.html) explain how to implement the operators in both [Scala](scala_api_guide.html) and [Java](java_api_guide.html#iterations). We also provide a **vertex-centric graph processing API** called [Spargel](spargel_guide.html).
+Here, we provide background on both operator variants and outline their usage. The [programming guide](programming_guide.html) explain how to implement the operators in both Scala and Java. We also provide a **vertex-centric graph processing API** called [Spargel](spargel_guide.html).
 
 The following table provides an overview of both operators:
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/97d630d5/docs/programming_guide.md
----------------------------------------------------------------------
diff --git a/docs/programming_guide.md b/docs/programming_guide.md
index 79a4820..883e769 100644
--- a/docs/programming_guide.md
+++ b/docs/programming_guide.md
@@ -11,7 +11,7 @@ title: "Flink Programming Guide"
 Introduction
 ------------
 
-Analysis programs in Flink are regular Java programs that implement transformations on data sets
+Analysis programs in Flink are regular programs that implement transformations on data sets
 (e.g., filtering, mapping, joining, grouping). The data sets are initially created from certain
 sources (e.g., by reading files, or from collections). Results are returned via sinks, which may for
 example write the data to (distributed) files, or to standard output (for example the command line
@@ -19,16 +19,18 @@ terminal). Flink programs run in a variety of contexts, standalone, or embedded
 The execution can happen in a local JVM, or on clusters of many machines.
 
 In order to create your own Flink program, we encourage you to start with the
-[program skeleton](#skeleton) and gradually add your own [transformations](#transformations).
-The remaining sections act as references for additional operations and advanced features.
+[program skeleton](#program-skeleton) and gradually add your own
+[transformations](#transformations). The remaining sections act as references for additional
+operations and advanced features.
 
 
 Example Program
 ---------------
 
 The following program is a complete, working example of WordCount. You can copy &amp; paste the code
-to run it locally. You only have to include Flink's Java API library into your project (see Section
-[Linking with Flink](#linking)) and specify the imports. Then you are ready to go!
+to run it locally. You only have to include the correct Flink's library into your project
+(see Section [Linking with Flink](#linking-with-flink)) and specify the imports. Then you are ready
+to go!
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -71,8 +73,9 @@ object WordCount {
   def main(args: Array[String]) {
 
     val env = ExecutionEnvironment.getExecutionEnvironment
-    val text = env.fromElements("Who's there?",
-            "I think I hear them. Stand, ho! Who's there?")
+    val text = env.fromElements(
+      "Who's there?",
+      "I think I hear them. Stand, ho! Who's there?")
 
     val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
       .map { (_, 1) }
@@ -95,22 +98,38 @@ object WordCount {
 Linking with Flink
 ------------------
 
-To write programs with Flink, you need to include Flink’s Java API library in your project.
+To write programs with Flink, you need to include the Flink library corresponding to
+your programming language in your project.
 
-The simplest way to do this is to use the [quickstart scripts](java_api_quickstart.html). They
+The simplest way to do this is to use one of the quickstart scripts: either for
+[Java](java_api_quickstart.html) or for [Scala](scala_api_quickstart.html). They
 create a blank project from a template (a Maven Archetype), which sets up everything for you. To
 manually create the project, you can use the archetype and create a project by calling:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight bash %}
 mvn archetype:generate /
     -DarchetypeGroupId=org.apache.flink/
     -DarchetypeArtifactId=flink-quickstart-java /
     -DarchetypeVersion={{site.FLINK_VERSION_STABLE }}
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight bash %}
+mvn archetype:generate /
+    -DarchetypeGroupId=org.apache.flink/
+    -DarchetypeArtifactId=flink-quickstart-scala /
+    -DarchetypeVersion={{site.FLINK_VERSION_STABLE }}
+{% endhighlight %}
+</div>
+</div>
 
 If you want to add Flink to an existing Maven project, add the following entry to your
 *dependencies* section in the *pom.xml* file of your project:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight xml %}
 <dependency>
   <groupId>org.apache.flink</groupId>
@@ -123,6 +142,22 @@ If you want to add Flink to an existing Maven project, add the following entry t
   <version>{{site.FLINK_VERSION_STABLE }}</version>
 </dependency>
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-scala</artifactId>
+  <version>{{site.FLINK_VERSION_STABLE }}</version>
+</dependency>
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients</artifactId>
+  <version>{{site.FLINK_VERSION_STABLE }}</version>
+</dependency>
+{% endhighlight %}
+</div>
+</div>
 
 If you are using Flink together with Hadoop, the version of the dependency may vary depending on the
 version of Hadoop (or more specifically, HDFS) that you want to use Flink with. Please refer to the
@@ -141,6 +176,9 @@ file and [run it on a cluster](cluster_execution.html), you can skip that depend
 Program Skeleton
 ----------------
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
 As we already saw in the example, Flink programs look like regular Java
 programs with a `main()` method. Each program consists of the same basic parts:
 
@@ -189,14 +227,14 @@ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 DataSet<String> text = env.readTextFile("file:///path/to/file");
 {% endhighlight %}
 
-This will give you a `DataSet` on which you can then apply transformations. For
+This will give you a DataSet on which you can then apply transformations. For
 more information on data sources and input formats, please refer to
 [Data Sources](#data_sources).
 
-Once you have a `DataSet` you can apply transformations to create a new
-`DataSet` which you can then write to a file, transform again, or
-combine with other `DataSet`s. You apply transformations by calling
-methods on `DataSet` with your own custom transformation function. For example,
+Once you have a DataSet you can apply transformations to create a new
+DataSet which you can then write to a file, transform again, or
+combine with other DataSets. You apply transformations by calling
+methods on DataSet with your own custom transformation function. For example,
 a map transformation looks like this:
 
 {% highlight java %}
@@ -210,12 +248,12 @@ DataSet<Integer> tokenized = text.map(new MapFunction<String, Integer>() {
 });
 {% endhighlight %}
 
-This will create a new `DataSet` by converting every String in the original
+This will create a new DataSet by converting every String in the original
 set to an Integer. For more information and a list of all the transformations,
 please refer to [Transformations](#transformations).
 
-Once you have a `DataSet` that needs to be written to disk you call one
-of these methods on `DataSet`:
+Once you have a DataSet that needs to be written to disk you call one
+of these methods on DataSet:
 
 {% highlight java %}
 writeAsText(String path)
@@ -226,14 +264,112 @@ print()
 {% endhighlight %}
 
 The last method is only useful for developing/debugging on a local machine,
-it will output the contents of the `DataSet` to standard output. (Note that in
+it will output the contents of the DataSet to standard output. (Note that in
+a cluster, the result goes to the standard out stream of the cluster nodes and ends
+up in the *.out* files of the workers).
+The first two do as the name suggests, the third one can be used to specify a
+custom data output format. Please refer
+to [Data Sinks](#data_sinks) for more information on writing to files and also
+about custom data output formats.
+
+Once you specified the complete program you need to call `execute` on
+the `ExecutionEnvironment`. This will either execute on your local
+machine or submit your program for execution on a cluster, depending on
+how you created the execution environment.
+
+</div>
+<div data-lang="scala" markdown="1">
+
+As we already saw in the example, Flink programs look like regular Scala
+programs with a `main()` method. Each program consists of the same basic parts:
+
+1. Obtain an `ExecutionEnvironment`,
+2. Load/create the initial data,
+3. Specify transformations on this data,
+4. Specify where to put the results of your computations, and
+5. Execute your program.
+
+We will now give an overview of each of those steps but please refer to the respective sections for
+more details. Note that all core classes of the Scala API are found in the package 
+{% gh_link /flink-scala/src/main/scala/org/apache/flink/api/scala "org.apache.flink.api.scala" %}.
+
+
+The `ExecutionEnvironment` is the basis for all Flink programs. You can
+obtain one using these static methods on class `ExecutionEnvironment`:
+
+{% highlight scala %}
+def getExecutionEnvironment
+
+def createLocalEnvironment(degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()))
+
+def createRemoteEnvironment(host: String, port: String, jarFiles: String*)
+def createRemoteEnvironment(host: String, port: String, degreeOfParallelism: Int, jarFiles: String*)
+{% endhighlight %}
+
+Typically, you only need to use `getExecutionEnvironment()`, since this
+will do the right thing depending on the context: if you are executing
+your program inside an IDE or as a regular Scala program it will create
+a local environment that will execute your program on your local machine. If
+you created a JAR file from you program, and invoke it through the [command line](cli.html)
+or the [web interface](web_client.html),
+the Flink cluster manager will
+execute your main method and `getExecutionEnvironment()` will return
+an execution environment for executing your program on a cluster.
+
+For specifying data sources the execution environment has several methods
+to read from files using various methods: you can just read them line by line,
+as CSV files, or using completely custom data input formats. To just read
+a text file as a sequence of lines, you can use:
+
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment()
+
+val text = env.readTextFile("file:///path/to/file")
+{% endhighlight %}
+
+This will give you a DataSet on which you can then apply transformations. For
+more information on data sources and input formats, please refer to
+[Data Sources](#data_sources).
+
+Once you have a DataSet you can apply transformations to create a new
+DataSet which you can then write to a file, transform again, or
+combine with other DataSets. You apply transformations by calling
+methods on DataSet with your own custom transformation function. For example,
+a map transformation looks like this:
+
+{% highlight scala %}
+val input: DataSet[String] = ...
+
+val mapped = text.map { x => x.toInt }
+{% endhighlight %}
+
+This will create a new DataSet by converting every String in the original
+set to an Integer. For more information and a list of all the transformations,
+please refer to [Transformations](#transformations).
+
+Once you have a DataSet that needs to be written to disk you can call one
+of these methods on DataSet:
+
+{% highlight scala %}
+def writeAsText(path: String, writeMode: WriteMode = WriteMode.NO_OVERWRITE)
+def writeAsCsv(
+    filePath: String,
+    rowDelimiter: String = "\n",
+    fieldDelimiter: String = ',',
+    writeMode: WriteMode = WriteMode.NO_OVERWRITE)
+def write(outputFormat: FileOutputFormat[T],
+    path: String,
+    writeMode: WriteMode = WriteMode.NO_OVERWRITE)
+
+def print()
+{% endhighlight %}
+
+The last method is only useful for developing/debugging on a local machine,
+it will output the contents of the DataSet to standard output. (Note that in
 a cluster, the result goes to the standard out stream of the cluster nodes and ends
 up in the *.out* files of the workers).
 The first two do as the name suggests, the third one can be used to specify a
-custom data output format. Keep in mind, that these calls do not actually
-write to a file yet. Only when your program is completely specified and you
-call the `execute` method on your `ExecutionEnvironment` are all the
-transformations executed and is data written to disk. Please refer
+custom data output format. Please refer
 to [Data Sinks](#data_sinks) for more information on writing to files and also
 about custom data output formats.
 
@@ -242,6 +378,10 @@ the `ExecutionEnvironment`. This will either execute on your local
 machine or submit your program for execution on a cluster, depending on
 how you created the execution environment.
 
+</div>
+</div>
+
+
 [Back to top](#top)
 
 
@@ -267,9 +407,14 @@ Data transformations transform one or more DataSets into a new DataSet. Programs
 multiple transformations into sophisticated assemblies.
 
 This section gives a brief overview of the available transformations. The [transformations
-documentation](java_api_transformations.html) has full description of all transformations with
+documentation](dataset_transformations.html) has a full description of all transformations with
 examples.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+<br />
+
 <table class="table table-bordered">
   <thead>
     <tr>
@@ -396,7 +541,7 @@ DataSet<Tuple3<Integer, String, Double>> output = input.sum(0).andMin(2);
         Joins two data sets by creating all pairs of elements that are equal on their keys.
         Optionally uses a JoinFunction to turn the pair of elements into a single element, or a
         FlatJoinFunction to turn the pair of elements into arbitararily many (including none)
-        elements. See <a href="#defining-keys">keys</a> on how to define join keys.
+        elements. See <a href="#specifying-keys">keys</a> on how to define join keys.
 {% highlight java %}
 result = input1.join(input2)
                .where(0)       // key of the first input (tuple field 0)
@@ -410,7 +555,7 @@ result = input1.join(input2)
       <td>
         <p>The two-dimensional variant of the reduce operation. Groups each input on one or more
         fields and then joins the groups. The transformation function is called per pair of groups.
-        See <a href="#defining-keys">keys</a> on how to define coGroup keys.</p>
+        See <a href="#specifying-keys">keys</a> on how to define coGroup keys.</p>
 {% highlight java %}
 data1.coGroup(data2)
      .where(0)
@@ -477,16 +622,170 @@ DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integ
   </tbody>
 </table>
 
-The [parallelism](#parallelism) of a transformation can be defined by `setParallelism(int)`.
+</div>
+<div data-lang="scala" markdown="1">
+<br />
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-left" style="width: 20%">Transformation</th>
+      <th class="text-center">Description</th>
+    </tr>
+  </thead>
+
+  <tbody>
+    <tr>
+      <td><strong>Map</strong></td>
+      <td>
+        <p>Takes one element and produces one element.</p>
+{% highlight scala %}
+data.map { x => x.toInt }
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>FlatMap</strong></td>
+      <td>
+        <p>Takes one element and produces zero, one, or more elements. </p>
+{% highlight scala %}
+data.flatMap { str => str.split(" ") }
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>MapPartition</strong></td>
+      <td>
+        <p>Transforms a parallel partition in a single function call. The function get the partition
+        as a `TraversableOnce` and can produce an arbitrary number of result values. The number of
+        elements in each partition depends on the degree-of-parallelism and previous operations.</p>
+{% highlight scala %}
+data.mapPartition { in => in map { (_, 1) } }
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Filter</strong></td>
+      <td>
+        <p>Evaluates a boolean function for each element and retains those for which the function
+        returns true.</p>
+{% highlight scala %}
+data.filter { _ > 1000 }
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Reduce</strong></td>
+      <td>
+        <p>Combines a group of elements into a single element by repeatedly combining two elements
+        into one. Reduce may be applied on a full data set, or on a grouped data set.</p>
+{% highlight scala %}
+data.reduce { _ + _ }
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>ReduceGroup</strong></td>
+      <td>
+        <p>Combines a group of elements into one or more elements. ReduceGroup may be applied on a
+        full data set, or on a grouped data set.</p>
+{% highlight scala %}
+data.reduceGroup { elements => elements.sum }
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Aggregate</strong></td>
+      <td>
+        <p>Aggregates a group of values into a single value. Aggregation functions can be thought of
+        as built-in reduce functions. Aggregate may be applied on a full data set, or on a grouped
+        data set.</p>
+{% highlight scala %}
+val input: DataSet[(Int, String, Double)] = // [...]
+val output: DataSet[(Int, String, Doublr)] = input.aggregate(SUM, 0).aggregate(MIN, 2);
+{% endhighlight %}
+  <p>You can also use short-hand syntax for minimum, maximum, and sum aggregations.</p>
+{% highlight scala %}
+val input: DataSet[(Int, String, Double)] = // [...]
+val output: DataSet[(Int, String, Doublr)] = input.sum(0).min(2)
+{% endhighlight %}
+      </td>
+    </tr>
+
+    </tr>
+      <td><strong>Join</strong></td>
+      <td>
+        Joins two data sets by creating all pairs of elements that are equal on their keys.
+        Optionally uses a JoinFunction to turn the pair of elements into a single element, or a
+        FlatJoinFunction to turn the pair of elements into arbitararily many (including none)
+        elements. See <a href="#specifying-keys">keys</a> on how to define join keys.
+{% highlight java %}
+// In this case tuple fields are used as keys. "0" is the join field on the first tuple
+// "1" is the join field on the second tuple.
+val result = input1.join(input2).where(0).equalTo(1)
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>CoGroup</strong></td>
+      <td>
+        <p>The two-dimensional variant of the reduce operation. Groups each input on one or more
+        fields and then joins the groups. The transformation function is called per pair of groups.
+        See <a href="#specifying-keys">keys</a> on how to define coGroup keys.</p>
+{% highlight java %}
+data1.coGroup(data2).where(0).equalTo(1)
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Cross</strong></td>
+      <td>
+        <p>Builds the Cartesian product (cross product) of two inputs, creating all pairs of
+        elements. Optionally uses a CrossFunction to turn the pair of elements into a single
+        element</p>
+{% highlight java %}
+val data1: DataSet[Int] = // [...]
+val data2: DataSet[String] = // [...]
+val result: DataSet[(Int, String)] = data1.cross(data2)
+{% endhighlight %}
+      </td>
+    </tr>
+    <tr>
+      <td><strong>Union</strong></td>
+      <td>
+        <p>Produces the union of two data sets.</p>
+{% highlight scala %}
+data.union(data2)
+{% endhighlight %}
+      </td>
+    </tr>
+  </tbody>
+</table>
+
+</div>
+</div>
+
+The [parallelism](#parallelism) of a transformation can be defined by `setParallelism(int)` while
 `name(String)` assigns a custom name to a transformation which is helpful for debugging. The same is
 possible for [Data Sources](#data_sources) and [Data Sinks](#data_sinks).
 
 [Back to Top](#top)
 
 
-Defining Keys
+Specifying Keys
 -------------
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
 Some transformations (join, coGroup) require that a key is defined on
 its argument DataSets, and other transformations (Reduce, GroupReduce,
 Aggregate) allow that the DataSet is grouped on a key before they are
@@ -515,8 +814,8 @@ DataSet<Tuple3<Integer,String,Long> grouped = input
 {% endhighlight %}
 
 The data set is grouped on the first field of the tuples (the one of
-Integer type). The GroupReduceFunction will thus receive groups with
-the same value of the first field.
+Integer type). The GroupReduceFunction will thus receive groups of tuples with
+the same value in the first field.
 
 {% highlight java %}
 DataSet<Tuple3<Integer,String,Long>> input = // [...]
@@ -570,14 +869,100 @@ DataSet<Tuple2<String, Double>>
                             });
 {% endhighlight %}
 
+</div>
+
+<div data-lang="scala" markdown="1">
+Some transformations (join, coGroup) require that a key is defined on
+its argument DataSets, and other transformations (Reduce, GroupReduce,
+Aggregate) allow that the DataSet is grouped on a key before they are
+applied.
+
+A DataSet is grouped as
+{% highlight scala %}
+val input: DataSet[...] = // [...]
+val reduced = input
+  .groupBy(/*define key here*/)
+  .reduceGroup(/*do something*/)
+{% endhighlight %}
+
+The data model of Flink is not based on key-value pairs. Therefore,
+you do not need to physically pack the data set types into keys and
+values. Keys are "virtual": they are defined as functions over the
+actual data to guide the grouping operator.
+
+The simplest case is grouping a data set of Case Classes on one or more
+of it's fields:
+{% highlight scala %}
+case class WordCount(docId: Int, word: String, count: Int)
+val input: DataSet[WordCount] = // [...]
+val grouped = input
+  .groupBy("word")
+  .reduceGroup(/*do something*/)
+{% endhighlight %}
+
+The data set is grouped on the second field of the Case Class (the one of
+String type). The group reduce function will thus receive groups of elements with
+the same value in the second field.
+
+{% highlight scala %}
+val input: DataSet[WordCount] = // [...]
+val grouped = input
+  .groupBy("docId", "word")
+  .reduceGroup(/*do something*/);
+{% endhighlight %}
+
+Here the DataSet is grouped on the composite key consisting of the first and the
+second fields, therefore the group reduce function will receive groups
+with the same value in both fields.
+
+As a special case, fields of Tuple DataSets can also be specified by (zero-based) index:
+{% highlight scala %}
+val input: DataSet[(Int, String, Int)] = // [...]
+val grouped = input
+  .groupBy(0, 1)
+  .reduceGroup(/*do something*/);
+{% endhighlight %}
+
+For DataSets that don't contain Case Classes or Tuples, key definition is done via a "key selector"
+function, which takes as argument one dataset element and must return a key of an
+arbitrary data type. For example:
+{% highlight scala %}
+// some ordinary object
+class WC {
+  val word: String
+  val count: Int
+}
+val words: DataSet[WC] = // [...]
+val counts: DataSet[WC] = words groupBy { _.word } reduce { /*do something*/}
+{% endhighlight %}
+
+Remember that keys are not only used for grouping, but also joining and matching data sets:
+{% highlight scala %}
+// some object
+case class Rating(name: String, category: String, points: Int)
+val ratings: DataSet[Rating] = // [...]
+val weights: DataSet[(String, Double)] = // [...]
+
+// Tuples are also Case Classes in Scala, so we could use this:
+val weightedRatings = ratings.join(weights).where("category").equalTo("_1")
+
+// Or This:
+val weightedRatings2 = ratings.join(weights).where("category").equalTo(0)
+{% endhighlight %}
+</div>
+</div>
+
+
 [Back to top](#top)
 
 
-Functions
----------
+Passing Functions to Flink
+--------------------------
 
-You can define a user-defined function and pass it to the DataSet
-transformations in several ways:
+Operations require user-defined functions. This section lists several ways for doing this.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
 #### Implementing an interface
 
@@ -618,18 +1003,23 @@ data.reduce((i1,i2) -> i1 + i2);
 
 All transformations that take as argument a user-defined function can
 instead take as argument a *rich* function. For example, instead of
+
 {% highlight java %}
 class MyMapFunction implements MapFunction<String, Integer> {
   public Integer map(String value) { return Integer.parseInt(value); }
 });
 {% endhighlight %}
+
 you can write
+
 {% highlight java %}
 class MyMapFunction extends RichMapFunction<String, Integer> {
   public Integer map(String value) { return Integer.parseInt(value); }
 });
 {% endhighlight %}
+
 and pass the function as usual to a `map` transformation:
+
 {% highlight java %}
 data.map(new MyMapFunction());
 {% endhighlight %}
@@ -641,6 +1031,59 @@ data.map (new RichMapFunction<String, Integer>() {
 });
 {% endhighlight %}
 
+</div>
+<div data-lang="scala" markdown="1">
+
+
+#### Lambda Functions
+
+As already seen in previous examples all operations accept lambda functions for describing
+the operation:
+{% highlight scala %}
+val data: DataSet[String] = // [...]
+data.filter { _.startsWith("http://") }
+{% endhighlight %}
+
+{% highlight scala %}
+val data: DataSet[Int] = // [...]
+data.reduce { (i1,i2) => i1 + i2 }
+// or
+data.reduce { _ + _ }
+{% endhighlight %}
+
+#### Rich functions
+
+All transformations that take as argument a lambda function can
+instead take as argument a *rich* function. For example, instead of
+
+{% highlight scala %}
+data.map { x => x.toInt }
+{% endhighlight %}
+
+you can write
+
+{% highlight scala %}
+class MyMapFunction extends RichMapFunction[String, Int] {
+  def map(in: String):Int = { in.toInt }
+})
+{% endhighlight %}
+
+and pass the function to a `map` transformation:
+
+{% highlight scala %}
+data.map(new MyMapFunction())
+{% endhighlight %}
+
+Rich functions can also be defined as an anonymous class:
+{% highlight scala %}
+data.map (new RichMapFunction[String, Int] {
+  def map(in: String):Int = { in.toInt }
+})
+{% endhighlight %}
+</div>
+
+</div>
+
 Rich functions provide, in addition to the user-defined function (map,
 reduce, etc), four methods: `open`, `close`, `getRuntimeContext`, and
 `setRuntimeContext`. These are useful for creating and finalizing
@@ -653,7 +1096,7 @@ on iterations (see [Iterations](#iterations)).
 In particular for the `reduceGroup` transformation, using a rich
 function is the only way to define an optional `combine` function. See
 the
-[transformations documentation](java_api_transformations.html)
+[transformations documentation](dataset_transformations.html)
 for a complete example.
 
 [Back to top](#top)
@@ -662,30 +1105,30 @@ for a complete example.
 Data Types
 ----------
 
-The Java API is strongly typed: All data sets and transformations accept typed elements. This
-catches type errors very early and supports safe refactoring of programs. The API supports various
-different data types for the input and output of operators. Both `DataSet` and functions like
-`MapFunction`, `ReduceFunction`, etc. are parameterized with data types using Java generics in order
-to ensure type-safety.
+Flink places some restrictions on the type of elements that are used in DataSets and as results
+of transformations. The reason for this is that the system analyzes the types to determine
+efficient execution strategies.
 
-There are four different categories of data types, which are treated slightly different:
+There are four different categories of data types, which are treated slightly different when it
+to [specifying keys](#specifying-keys):
 
-1. **Regular Types**
-2. **Tuples**
+1. **General Types**
+2. **Tuples**/**Case Classes**
 3. **Values**
 4. **Hadoop Writables**
 
 
-#### Regular Types
+#### General Types
 
-Out of the box, the Java API supports all common basic Java types: `Byte`, `Short`, `Integer`,
-`Long`, `Float`, `Double`, `Boolean`, `Character`, `String`.
+Out of the box, Flink supports all primitive types of your programming language of choice.
 
-Furthermore, you can use the vast majority of custom Java classes. Restrictions apply to classes
+Furthermore, you can use the vast majority of custom classes. Restrictions apply to classes
 containing fields that cannot be serialized, like File pointers, I/O streams, or other native
 resources. Classes that follow the Java Beans conventions work well in general. The following
 defines a simple example class to illustrate how you can use custom classes:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 public class WordWithCount {
 
@@ -700,11 +1143,23 @@ public class WordWithCount {
     }
 }
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+class WordWithCount(val word: String, val count: Int) {
+    def this() {
+      this(null, -1)
+    }
+}
+{% endhighlight %}
+</div>
+</div>
 
-You can use all of those types to parameterize `DataSet` and function implementations, e.g.
-`DataSet<String>` for a `String` data set or `MapFunction<String, Integer>` for a mapper from
-`String` to `Integer`.
+You can use all of those types to parameterize DataSet and function implementations, e.g.
+`DataSet` or a `MapFunction`.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 // using a basic data type
 DataSet<String> numbers = env.fromElements("1", "2");
@@ -728,11 +1183,37 @@ wordCounts.map(new MapFunction<WordCount, Integer>() {
     }
 });
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+// using a primitive data type
+// Note that the type ascription "DataSet[String]" can be omitted in Scala
+// it is just given to clarify the type of numbers
+val numbers: DataSet[String] = env.fromElements("1", "2")
+
+numbers.map(new MapFunction[String, Int]() {
+    def map(in: String): Int = {
+      in.toInt
+    }
+})
+
+// using a custom class
+val wordCounts = env.fromElements(
+  new WordCount("hello", 1),
+  new WordCount("world", 2))
+
+wordCounts.map { _.count }
+{% endhighlight %}
+</div>
+</div>
+
 
 When working with operators that require a Key for grouping or matching records
-you need to implement a `KeySelector` for your custom type (see
-[Defining Keys](#defining-keys)).
+you need to implement a key selector function for your custom type (see
+[Specifying Keys](#specifying-keys)).
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 wordCounts.groupBy(new KeySelector<WordCount, String>() {
     public String getKey(WordCount v) {
@@ -740,8 +1221,19 @@ wordCounts.groupBy(new KeySelector<WordCount, String>() {
     }
 }).reduce(new MyReduceFunction());
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+wordCounts groupBy { _.word } reduce(new MyReduceFunction())
+{% endhighlight %}
+</div>
+</div>
+
 
-#### Tuples
+#### Tuples/Case Classes
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
 You can use the `Tuple` classes for composite types. Tuples contain a fix number of fields of
 various types. The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple
@@ -779,21 +1271,46 @@ the field positions. See this
 {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java "example" %} for an
 illustration how to make use of that mechanism.
 
+</div>
+<div data-lang="scala" markdown="1">
+
+Flink has special support for Scala's Case Classes and Tuples. When using working with an operator
+that required a key for grouping or matching records this key can be specified using tuple field
+positions or field names: 
+
+{% highlight scala %}
+case class WordCount(word: String, count: Int)
+val input = env.fromElements(
+    WordCount("hello", 1),
+    WordCount("world", 2))
+
+input.groupBy("word").reduce(...)
+
+val input2 = env.fromElements(("hello", 1), ("world", 2))
+
+input2.groupBy(0, 1).reduce(...)
+{% endhighlight %}
+
+Both variants allow specifying more than one key field name or key field position. See
+[specifying keys](#specifying-keys) for more details.
+
+</div>
+</div>
 
 #### Values
 
-*Value types describe their serialization and deserialization manually. Instead of going through a
-*general purpose serialization framework, they provide custom code for those operations by means
-*implementing the `org.apache.flinktypes.Value` interface with the methods `read` and `write`. Using
-*a Value type is reasonable when general purpose serialization would be highly inefficient. An
-*example would be a data type that implements a sparse vector of elements as an array. Knowing that
-*the array is mostly zero, one can use a special encoding for the non-zero elements, while the
-*general purpose serialization would simply write all array elements.
+*Value* types describe their serialization and deserialization manually. Instead of going through a
+general purpose serialization framework, they provide custom code for those operations by means of
+implementing the `org.apache.flinktypes.Value` interface with the methods `read` and `write`. Using
+a Value type is reasonable when general purpose serialization would be highly inefficient. An
+example would be a data type that implements a sparse vector of elements as an array. Knowing that
+the array is mostly zero, one can use a special encoding for the non-zero elements, while the
+general purpose serialization would simply write all array elements.
 
 The `org.apache.flinktypes.CopyableValue` interface supports manual internal cloning logic in a
 similar way.
 
-Flink comes with pre-defined Value types that correspond to Java's basic data types. (`ByteValue`,
+Flink comes with pre-defined Value types that correspond to basic data types. (`ByteValue`,
 `ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`,
 `BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can
 be altered, allowing programmers to reuse objects and take pressure off the garbage collector.
@@ -807,7 +1324,9 @@ defined in the `write()`and `readFields()` methods will be used for serializatio
 
 #### Type Erasure & Type Inferrence
 
-The Java compiler throws away much of the generic type information after the compilation. This is
+*Note: This Section is only relevant for Java.*
+
+The Java compiler throws away much of the generic type information after compilation. This is
 known as *type erasure* in Java. It means that at runtime, an instance of an object does not know
 its generic type any more. For example, instances of `DataSet<String>` and `DataSet<Long>` look the
 same to the JVM.
@@ -832,8 +1351,92 @@ usually be inferred by the result types of the previous operations.
 [Back to top](#top)
 
 
-Data Sources
-------------
+Data Sources
+------------
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+Data sources create the initial data sets, such as from files or from Java collections. The general
+mechanism of of creating data sets is abstracted behind an
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java "InputFormat"%}.
+Flink comes
+with several built-in formats to create data sets from common file formats. Many of them have
+shortcut methods on the *ExecutionEnvironment*.
+
+File-based:
+
+- `readTextFile(path)` / `TextInputFormat` - Reads files line wise and returns them as Strings.
+- `readTextFileWithValue(path)` / `TextValueInputFormat` - Reads files line wise and returns them as
+  StringValues. StringValues are mutable strings.
+- `readCsvFile(path)` / `CsvInputFormat` - Parses files of comma (or another char) delimited fields.
+  Returns a DataSet of tuples. Supports the basic java types and their Value counterparts as field
+  types.
+
+Collection-based:
+
+- `fromCollection(Collection)` - Creates a data set from the Java Java.util.Collection. All elements
+  in the collection must be of the same type.
+- `fromCollection(Iterator, Class)` - Creates a data set from an iterator. The class specifies the
+  data type of the elements returned by the iterator.
+- `fromElements(T ...)` - Creates a data set from the given sequence of objects. All objects must be
+  of the same type.
+- `fromParallelCollection(SplittableIterator, Class)` - Creates a data set from an iterator, in
+  parallel. The class specifies the data type of the elements returned by the iterator.
+- `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in
+  parallel.
+
+Generic:
+
+- `readFile(inputFormat, path)` / `FileInputFormat` - Accepts a file input format.
+- `createInput(inputFormat)` / `InputFormat` - Accepts a generic input format.
+
+**Examples**
+
+{% highlight java %}
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// read text file from local files system
+DataSet<String> localLines = env.readTextFile("file:///path/to/my/textfile");
+
+// read text file from a HDFS running at nnHost:nnPort
+DataSet<String> hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile");
+
+// read a CSV file with three fields
+DataSet<Tuple3<Integer, String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+	                       .types(Integer.class, String.class, Double.class);
+
+// read a CSV file with five fields, taking only two of them
+DataSet<Tuple2<String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+                               .includeFields("10010")  // take the first and the fourth field
+	                       .types(String.class, Double.class);
+
+// create a set from some given elements
+DataSet<String> value = env.fromElements("Foo", "bar", "foobar", "fubar");
+
+// generate a number sequence
+DataSet<Long> numbers = env.generateSequence(1, 10000000);
+
+// Read data from a relational database using the JDBC input format
+DataSet<Tuple2<String, Integer> dbData = 
+    env.createInput(
+      // create and configure input format
+      JDBCInputFormat.buildJDBCInputFormat()
+                     .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                     .setDBUrl("jdbc:derby:memory:persons")
+                     .setQuery("select name, age from persons")
+                     .finish(),
+      // specify type information for DataSet
+      new TupleTypeInfo(Tuple2.class, STRING_TYPE_INFO, INT_TYPE_INFO)
+    );
+
+// Note: Flink's program compiler needs to infer the data types of the data items which are returned
+// by an InputFormat. If this information cannot be automatically inferred, it is necessary to
+// manually provide the type information as shown in the examples above.
+{% endhighlight %}
+
+</div>
+<div data-lang="scala" markdown="1">
 
 Data sources create the initial data sets, such as from files or from Java collections. The general
 mechanism of of creating data sets is abstracted behind an
@@ -853,69 +1456,64 @@ File-based:
 
 Collection-based:
 
-- `fromCollection(Collection)` - Creates a data set from the Java Java.util.Collection. All elements
+- `fromCollection(Seq)` - Creates a data set from a Seq. All elements
   in the collection must be of the same type.
-- `fromCollection(Iterator, Class)` - Creates a data set from an iterator. The class specifies the
+- `fromCollection(Iterator)` - Creates a data set from an Iterator. The class specifies the
   data type of the elements returned by the iterator.
-- `fromElements(T ...)` - Creates a data set from the given sequence of objects. All objects must be
-  of the same type.
-- `fromParallelCollection(SplittableIterator, Class)` - Creates a data set from an iterator, in
+- `fromElements(elements: _*)` - Creates a data set from the given sequence of objects. All objects
+  must be of the same type.
+- `fromParallelCollection(SplittableIterator)` - Creates a data set from an iterator, in
   parallel. The class specifies the data type of the elements returned by the iterator.
 - `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in
   parallel.
 
 Generic:
 
-- `createInput(path)` / `InputFormat` - Accepts a generic input format.
+- `readFile(inputFormat, path)` / `FileInputFormat` - Accepts a file input format.
+- `createInput(inputFormat)` / `InputFormat` - Accepts a generic input format.
 
 **Examples**
 
-{% highlight java %}
-ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+{% highlight scala %}
+val env  = ExecutionEnvironment.getExecutionEnvironment
 
 // read text file from local files system
-DataSet<String> localLines = env.readTextFile("file:///path/to/my/textfile");
+val localLiens = env.readTextFile("file:///path/to/my/textfile")
 
 // read text file from a HDFS running at nnHost:nnPort
-DataSet<String> hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile");
+val hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile")
 
 // read a CSV file with three fields
-DataSet<Tuple3<Integer, String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
-	                       .types(Integer.class, String.class, Double.class);
+val csvInput = env.readCsvFile[(Int, String, Double)]("hdfs:///the/CSV/file")
 
 // read a CSV file with five fields, taking only two of them
-DataSet<Tuple2<String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
-                               .includeFields("10010")  // take the first and the fourth fild
-	                       .types(String.class, Double.class);
+val csvInput = env.readCsvFile[(String, Double)](
+  "hdfs:///the/CSV/file",
+  includedFields = Array(0, 3)) // take the first and the fourth field
+
+// CSV input can also be used with Case Classes
+case class MyInput(str: String, dbl: Double)
+val csvInput = env.readCsvFile[MyInput](
+  "hdfs:///the/CSV/file",
+  includedFields = Array(0, 3)) // take the first and the fourth field
 
 // create a set from some given elements
-DataSet<String> value = env.fromElements("Foo", "bar", "foobar", "fubar");
+val values = env.fromElements("Foo", "bar", "foobar", "fubar")
 
 // generate a number sequence
-DataSet<Long> numbers = env.generateSequence(1, 10000000);
-
-// Read data from a relational database using the JDBC input format
-DataSet<Tuple2<String, Integer> dbData = 
-    env.createInput(
-      // create and configure input format
-      JDBCInputFormat.buildJDBCInputFormat()
-                     .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
-                     .setDBUrl("jdbc:derby:memory:persons")
-                     .setQuery("select name, age from persons")
-                     .finish(),
-      // specify type information for DataSet
-      new TupleTypeInfo(Tuple2.class, STRING_TYPE_INFO, INT_TYPE_INFO)
-    );
-
-// Note: Flink's program compiler needs to infer the data types of the data items which are returned
-// by an InputFormat. If this information cannot be automatically inferred, it is necessary to
-// manually provide the type information as shown in the examples above. {% endhighlight %}
+val numbers = env.generateSequence(1, 10000000);
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
 Data Sinks
 ----------
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
 Data sinks consume DataSets and are used to store or return them. Data sink operations are described
 using an
 {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java "OutputFormat" %}.
@@ -926,7 +1524,7 @@ DataSet:
   obtained by calling the *toString()* method of each element.
 - `writeAsFormattedText()` / `TextOutputFormat` - Write elements line-wise as Strings. The Strings
   are obtained by calling a user-defined *format()* method for each element.
-- `writeAsCsv` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field
+- `writeAsCsv(...)` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field
   delimiters are configurable. The value for each field comes from the *toString()* method of the objects.
 - `print()` / `printToErr()` - Prints the *toString()* value of each element on the
   standard out / strandard error stream.
@@ -987,8 +1585,61 @@ myResult.output(
     );
 {% endhighlight %}
 
-[Back to top](#top)
+</div>
+<div data-lang="scala" markdown="1">
+Data sinks consume DataSets and are used to store or return them. Data sink operations are described
+using an
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java "OutputFormat" %}.
+Flink comes with a variety of built-in output formats that are encapsulated behind operations on the
+DataSet:
+
+- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are
+  obtained by calling the *toString()* method of each element.
+- `writeAsCsv(...)` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field
+  delimiters are configurable. The value for each field comes from the *toString()* method of the objects.
+- `print()` / `printToErr()` - Prints the *toString()* value of each element on the
+  standard out / strandard error stream.
+- `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports
+  custom object-to-bytes conversion.
+- `output()`/ `OutputFormat` - Most generic output method, for data sinks that are not file based
+  (such as storing the result in a database).
+
+A DataSet can be input to multiple operations. Programs can write or print a data set and at the
+same time run additional transformations on them.
+
+**Examples**
+
+Standard data sink methods:
+
+{% highlight scala %}
+// text data 
+val textData: DataSet[String] = // [...]
+
+// write DataSet to a file on the local file system
+textData.writeAsText("file:///my/result/on/localFS")
+
+// write DataSet to a file on a HDFS with a namenode running at nnHost:nnPort
+textData.writeAsText("hdfs://nnHost:nnPort/my/result/on/localFS")
+
+// write DataSet to a file and overwrite the file if it exists
+textData.writeAsText("file:///my/result/on/localFS", WriteMode.OVERWRITE)
+
+// tuples as lines with pipe as the separator "a|b|c"
+val values: DataSet[(String, Int, Double)] = // [...]
+values.writeAsCsv("file:///path/to/the/result/file", "\n", "|")
+
+// this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines
+values.writeAsText("file:///path/to/the/result/file");
+
+// this wites values as strings using a user-defined formatting
+values map { tuple => tuple._1 + " - " + tuple._2 }
+  .writeAsText("file:///path/to/the/result/file")
+{% endhighlight %}
+
+</div>
+</div>
 
+[Back to top](#top)
 
 Debugging
 ---------
@@ -1008,6 +1659,9 @@ start the LocalEnvironement from an IDE, you can set breakpoint in your code and
 program.
 
 A LocalEnvironment is created and used as follows:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
 
@@ -1015,18 +1669,32 @@ DataSet<String> lines = env.readTextFile(pathToTextFile);
 // build your program
 
 env.execute();
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
 
+{% highlight scala %}
+val env = ExecutionEnvironment.createLocalEnvironment()
+
+val lines = env.readTextFile(pathToTextFile)
+// build your program
+
+env.execute();
 {% endhighlight %}
+</div>
+</div>
 
 ### Collection Data Sources and Sinks
 
-Providing input for an analysis program and checking its output is cumbersome done by creating input
-files and reading output files. Flink features special data sources and sinks which are backed by
-Java collections to ease testing. Once a program has been tested, the sources and sinks can be
+Providing input for an analysis program and checking its output is cumbersome when done by creating
+input files and reading output files. Flink features special data sources and sinks which are backed
+by Java collections to ease testing. Once a program has been tested, the sources and sinks can be
 easily replaced by sources and sinks that read from / write to external data stores such as HDFS.
 
 Collection data sources can be used as follows:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
 
@@ -1042,10 +1710,6 @@ Iterator<Long> longIt = ...
 DataSet<Long> myLongs = env.fromCollection(longIt, Long.class);
 {% endhighlight %}
 
-**Note: Currently, the collection data source requires that data types and iterators implement
-**`Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of
-**parallelism = 1).
-
 A collection data sink is specified as follows:
 
 {% highlight java %}
@@ -1057,8 +1721,30 @@ myResult.output(new LocalCollectionOutputFormat(outData));
 
 **Note:** Currently, the collection data sink is restricted to local execution, as a debugging tool.
 
-[Back to top](#top)
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.createLocalEnvironment()
+
+// Create a DataSet from a list of elements
+val myInts = env.fromElements(1, 2, 3, 4, 5)
 
+// Create a DataSet from any Collection
+val data: Seq[(String, Int)] = ...
+val myTuples = env.fromCollection(data)
+
+// Create a DataSet from an Iterator
+val longIt: Iterator[Long] = ...
+val myLongs = env.fromCollection(longIt)
+{% endhighlight %}
+</div>
+</div>
+
+**Note:** Currently, the collection data source requires that data types and iterators implement
+`Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of
+parallelism = 1).
+
+[Back to top](#top)
 
 Iteration Operators
 -------------------
@@ -1071,9 +1757,12 @@ into the next iteration. There are two types of iterations in Flink: **BulkItera
 This section provides quick examples on how to use both operators. Check out the [Introduction to
 Iterations](iterations.html) page for a more detailed introduction.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
 #### Bulk Iterations
 
-To create a BulkIteration call the `iterate(int)` method of the `DataSet` the iteration should start
+To create a BulkIteration call the `iterate(int)` method of the DataSet the iteration should start
 at. This will return an `IterativeDataSet`, which can be transformed with the regular operators. The
 single argument to the iterate call specifies the maximum number of iterations.
 
@@ -1173,6 +1862,98 @@ iteration.closeWith(deltas, nextWorkset)
 	.writeAsCsv(outputPath);
 {% endhighlight %}
 
+</div>
+<div data-lang="scala" markdown="1">
+#### Bulk Iterations
+
+To create a BulkIteration call the `iterate(int)` method of the DataSet the iteration should start
+at and also specify a step function. The step function gets the input DataSet for the current
+iteration and must return a new DataSet. The parameter of the iterate call is the maximum number
+of iterations after which to stop.
+
+There is also the `iterateWithTermination(int)` function that accepts a step function that
+returns two DataSets: The result of the iteration step and a termination criterion. The iterations
+are stopped once the termination criterion DataSet is empty.
+
+The following example iteratively estimates the number Pi. The goal is to count the number of random
+points, which fall into the unit circle. In each iteration, a random point is picked. If this point
+lies inside the unit circle, we increment the count. Pi is then estimated as the resulting count
+divided by the number of iterations multiplied by 4.
+
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment()
+
+// Create initial DataSet
+val initial = env.fromElements(0)
+
+val count = initial.iterate(10000) { iterationInput: DataSet[Int] =>
+  val result = iterationInput.map { i => 
+    val x = Math.random()
+    val y = Math.randon()
+    i + (if (x * x + y * y < 1) 1 else 0)
+  }
+  result
+}
+
+val result = count map { c => c / 10000.0 * 4 }
+
+result.print()
+
+env.execute("Iterative Pi Example");
+{% endhighlight %}
+
+You can also check out the
+{% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala "K-Means example" %},
+which uses a BulkIteration to cluster a set of unlabeled points.
+
+#### Delta Iterations
+
+Delta iterations exploit the fact that certain algorithms do not change every data point of the
+solution in each iteration.
+
+In addition to the partial solution that is fed back (called workset) in every iteration, delta
+iterations maintain state across iterations (called solution set), which can be updated through
+deltas. The result of the iterative computation is the state after the last iteration. Please refer
+to the [Introduction to Iterations](iterations.html) for an overview of the basic principle of delta
+iterations.
+
+Defining a DeltaIteration is similar to defining a BulkIteration. For delta iterations, two data
+sets form the input to each iteration (workset and solution set), and two data sets are produced as
+the result (new workset, solution set delta) in each iteration.
+
+To create a DeltaIteration call the `iterateDelta(initialWorkset, maxIterations, key)` on the
+initial solution set. The step function takes two parameters: (solutionSet, workset), and must
+return two values: (solutionSetDelta, newWorkset).
+
+Below is an example for the syntax of a delta iteration
+
+{% highlight scala %}
+// read the initial data sets
+val initialSolutionSet: DataSet[(Long, Double)] = // [...]
+
+val initialWorkset: DataSet[(Long, Double)] = // [...]
+
+val maxIterations = 100
+val keyPosition = 0
+
+val result = initialSolutionSet.iterateDelta(initialWorkset, maxIterations, Array(keyPosition)) {
+  (solution, workset) =>
+    val candidateUpdates = workset.groupBy(1).reduceGroup(new ComputeCandidateChanges())
+    val deltas = candidateUpdates.join(solution).where(0).equalTo(0)(new CompareChangesToCurrent())
+
+    val nextWorkset = deltas.filter(new FilterByThreshold())
+
+    (deltas, nextWorkset)
+}
+
+result.writeAsCsv(outputPath)
+
+env.execute()
+{% endhighlight %}
+
+</div>
+</div>
+
 [Back to top](#top)
 
 
@@ -1186,9 +1967,11 @@ allow the system to reason about reusing sort orders or partitions across multip
 semantic annotations may eventually save the program from unnecessary data shuffling or unnecessary
 sorts.
 
-Semantic annotations can be attached to functions through Java annotations, or passed as arguments
+Semantic annotations can be attached to functions through Annotations, or passed as arguments
 when invoking a function on a DataSet. The following example illustrates that:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 @ConstantFields("1")
 public class DivideFirstbyTwo implements MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
@@ -1199,6 +1982,20 @@ public class DivideFirstbyTwo implements MapFunction<Tuple2<Integer, Integer>, T
   }
 }
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+@ConstantFields("1")
+class DivideFirstbyTwo extends MapFunction[(Int, Int), (Int, Int)]{
+   def map(input: (Int, Int): (Int, Int) = {
+    value.f0 /= 2;
+    (input._1 / 2, input._2)
+  }
+}
+{% endhighlight %}
+
+</div>
+</div>
 
 The following annotations are currently available:
 
@@ -1220,8 +2017,8 @@ The following annotations are currently available:
 * `@ConstantFieldsSecondExcept`: Declares that all fields of the second input are constant, except
   for the specified fields. Applicable to functions with a two input data sets.
 
-*(Note: The system currently evaluated annotations only on Tuple data types. 
-his will be extended in the next versions)*
+*(Note: The system currently only evaluates annotations only Tuple DataSets.  This will be extended
+in the next versions)*
 
 **Note**: It is important to be conservative when providing annotations. Only annotate fields,
 when they are always constant for every call to the function. Otherwise the system has incorrect
@@ -1236,12 +2033,14 @@ Broadcast Variables
 
 Broadcast variables allow you to make a data set available to all parallel instances of an
 operation, in addition to the regular input of the operation. This is useful for auxiliary data
-sets, or data-dependent parameterization. The data set will then be accessible at the operator as an
-`Collection<T>`.
+sets, or data-dependent parameterization. The data set will then be accessible at the operator as a
+Collection.
 
 - **Broadcast**: broadcast sets are registered by name via `withBroadcastSet(DataSet, String)`, and
 - **Access**: accessible via `getRuntimeContext().getBroadcastVariable(String)` at the target operator.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 // 1. The DataSet to be broadcasted
 DataSet<Integer> toBroadcast = env.fromElements(1, 2, 3);
@@ -1266,6 +2065,34 @@ data.map(new MapFunction<String, String>() {
 Make sure that the names (`broadcastSetName` in the previous example) match when registering and
 accessing broadcasted data sets. For a complete example program, have a look at
 {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java#L96 "KMeans Algorithm" %}.
+</div>
+<div data-lang="scala" markdown="1">
+
+{% highlight scala %}
+// 1. The DataSet to be broadcasted
+val toBroadcast = env.fromElements(1, 2, 3)
+
+val data = env.fromElements("a", "b")
+
+data.map(new RichMapFunction[String, String]() {
+    var broadcastSet: Traversable[String] = null
+
+    override def open(config: Configuration): Unit = {
+      // 3. Access the broadcasted DataSet as a Collection
+      broadcastSet = getRuntimeContext().getBroadcastVariable[String]("broadcastSetName").asScala
+    }
+
+    def map(in: String): String = {
+        ...
+    }
+}).withBroadcastSet(toBroadcast, "broadcastSetName") // 2. Broadcast the DataSet
+{% endhighlight %}
+
+Make sure that the names (`broadcastSetName` in the previous example) match when registering and
+accessing broadcasted data sets. For a complete example program, have a look at
+{% gh_link /flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala#L96 "KMeans Algorithm" %}.
+</div>
+</div>
 
 **Note**: As the content of broadcast variables is kept in-memory on each node, it should not become
 too large. For simpler things like scalar values you can simply make parameters part of the closure
@@ -1291,8 +2118,8 @@ interface. If the Flink program is invoked differently than through these interf
 environment will act like a local environment.
 
 To package the program, simply export all involved classes as a JAR file. The JAR file's manifest
-must point to the class that contains the program's *entry point* (the class with the `public void
-main(String[])` method). The simplest way to do this is by putting the *main-class* entry into the
+must point to the class that contains the program's *entry point* (the class with the public
+`main` method). The simplest way to do this is by putting the *main-class* entry into the
 manifest (such as `main-class: org.apache.flinkexample.MyProgram`). The *main-class* attribute is
 the same one that is used by the Java Virtual Machine to find the main method when executing a JAR
 files through the command `java -jar pathToTheJarFile`. Most IDEs offer to include that attribute
@@ -1301,11 +2128,11 @@ automatically when exporting JAR files.
 
 #### Packaging Programs through Plans
 
-Additionally, the Java API supports packaging programs as *Plans*. This method resembles the way
-that the *Scala API* packages programs. Instead of defining a progam in the main method and calling
+Additionally, we support packaging programs as *Plans*. Instead of defining a progam in the main
+method and calling
 `execute()` on the environment, plan packaging returns the *Program Plan*, which is a description of
 the program's data flow. To do that, the program must implement the
-`org.apache.flinkapi.common.Program` interface, defining the `getPlan(String...)` method. The
+`org.apache.flink.api.common.Program` interface, defining the `getPlan(String...)` method. The
 strings passed to that method are the command line arguments. The program's plan can be created from
 the environment via the `ExecutionEnvironment#createProgramPlan()` method. When packaging the
 program's plan, the JAR manifest must point to the class implementing the
@@ -1339,8 +2166,8 @@ which is available after the job ended.
 
 The most straightforward accumulator is a **counter**: You can increment it using the
 ```Accumulator.add(V value)``` method. At the end of the job Flink will sum up (merge) all partial
-results and send the result to the client. Since accumulators are very easy to use, they can be
-useful during debugging or if you quickly want to find out more about your data.
+results and send the result to the client. Accumulators are useful during debugging or if you
+quickly want to find out more about your data.
 
 Flink currently has the following **built-in accumulators**. Each of them implements the
 {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
@@ -1431,6 +2258,8 @@ The parallelism of an individual operator, data source, or data sink can be defi
 [WordCount](#example) example program can be set to `5` as follows :
 
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -1443,6 +2272,22 @@ wordCounts.print();
 
 env.execute("Word Count Example");
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+val text = [...]
+val wordCounts = text
+    .flatMap{ _.split(" ") map { (_, 1) } }
+    .groupBy(0)
+    .sum(1).setParallelism(5)
+wordCounts.print()
+
+env.execute("Word Count Example")
+{% endhighlight %}
+</div>
+</div>
 
 ### Execution Environment Level
 
@@ -1452,10 +2297,12 @@ it executes. Execution environment parallelism can be overwritten by explicitly
 parallelism of an operator.
 
 The default parallelism of an execution environment can be specified by calling the
-`setDefaultLocalParallelism()` method. To execute all operators, data sources, and data sinks of the
+`setDegreeOfParallelism()` method. To execute all operators, data sources, and data sinks of the
 [WordCount](#example) example program with a parallelism of `3`, set the default parallelism of the
 execution environment as follows:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 env.setDegreeOfParallelism(3);
@@ -1466,6 +2313,23 @@ wordCounts.print();
 
 env.execute("Word Count Example");
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+env.setDegreeOfParallelism(3)
+
+val text = [...]
+val wordCounts = text
+    .flatMap{ _.split(" ") map { (_, 1) } }
+    .groupBy(0)
+    .sum(1)
+wordCounts.print()
+
+env.execute("Word Count Example")
+{% endhighlight %}
+</div>
+</div>
 
 ### System Level
 
@@ -1484,13 +2348,15 @@ useful to know how exactly Flink will execute your program.
 
 __Plan Visualization Tool__
 
-Flink 0.5 comes packaged with a visualization tool for execution plans. The HTML document containing
+Flink comes packaged with a visualization tool for execution plans. The HTML document containing
 the visualizer is located under ```tools/planVisualizer.html```. It takes a JSON representation of
 the job execution plan and visualizes it as a graph with complete annotations of execution
 strategies.
 
 The following code shows how to print the execution plan JSON from your program:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -1498,6 +2364,17 @@ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
 System.out.println(env.getExecutionPlan());
 {% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+...
+
+println(env.getExecutionPlan())
+{% endhighlight %}
+</div>
+</div>
 
 
 To visualize the execution plan, do the following:


[22/60] Renamed java examples package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java
new file mode 100644
index 0000000..2d9fadc
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.wordcount.util;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * Provides the default data sets used for the WordCount example program.
+ * The default data sets are used, if no parameters are given to the program.
+ *
+ */
+public class WordCountData {
+
+	public static final String[] WORDS = new String[] {
+		"To be, or not to be,--that is the question:--",
+		"Whether 'tis nobler in the mind to suffer",
+		"The slings and arrows of outrageous fortune",
+		"Or to take arms against a sea of troubles,",
+		"And by opposing end them?--To die,--to sleep,--",
+		"No more; and by a sleep to say we end",
+		"The heartache, and the thousand natural shocks",
+		"That flesh is heir to,--'tis a consummation",
+		"Devoutly to be wish'd. To die,--to sleep;--",
+		"To sleep! perchance to dream:--ay, there's the rub;",
+		"For in that sleep of death what dreams may come,",
+		"When we have shuffled off this mortal coil,",
+		"Must give us pause: there's the respect",
+		"That makes calamity of so long life;",
+		"For who would bear the whips and scorns of time,",
+		"The oppressor's wrong, the proud man's contumely,",
+		"The pangs of despis'd love, the law's delay,",
+		"The insolence of office, and the spurns",
+		"That patient merit of the unworthy takes,",
+		"When he himself might his quietus make",
+		"With a bare bodkin? who would these fardels bear,",
+		"To grunt and sweat under a weary life,",
+		"But that the dread of something after death,--",
+		"The undiscover'd country, from whose bourn",
+		"No traveller returns,--puzzles the will,",
+		"And makes us rather bear those ills we have",
+		"Than fly to others that we know not of?",
+		"Thus conscience does make cowards of us all;",
+		"And thus the native hue of resolution",
+		"Is sicklied o'er with the pale cast of thought;",
+		"And enterprises of great pith and moment,",
+		"With this regard, their currents turn awry,",
+		"And lose the name of action.--Soft you now!",
+		"The fair Ophelia!--Nymph, in thy orisons",
+		"Be all my sins remember'd."
+	};
+
+	public static DataSet<String> getDefaultTextLineDataSet(ExecutionEnvironment env) {
+		return env.fromElements(WORDS);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
index b6ec64b..a0bb874 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
@@ -20,7 +20,7 @@ package org.apache.flink.examples.scala.clustering
 import org.apache.flink.api.common.functions._
 import org.apache.flink.api.scala._
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.example.java.clustering.util.KMeansData
+import org.apache.flink.examples.java.clustering.util.KMeansData
 
 import scala.collection.JavaConverters._
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
index f440b6f..8e3fbfd 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
@@ -15,56 +15,41 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.examples.scala.graph;
+package org.apache.flink.examples.scala.graph
 
 import org.apache.flink.api.scala._
-import org.apache.flink.example.java.graph.util.ConnectedComponentsData
-import org.apache.flink.api.java.aggregation.Aggregations
-import org.apache.flink.util.Collector
+import org.apache.flink.examples.java.graph.util.ConnectedComponentsData
 
 /**
  * An implementation of the connected components algorithm, using a delta iteration.
- * 
- * Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the minimum of its own ID and its
- * neighbors' IDs, as its new ID and tells its neighbors about its new ID. After the algorithm has completed, all vertices in the
- * same component will have the same ID.
- * 
- * A vertex whose component ID did not change needs not propagate its information in the next step. Because of that,
- * the algorithm is easily expressible via a delta iteration. We here model the solution set as the vertices with
- * their current component ids, and the workset as the changed vertices. Because we see all vertices initially as
- * changed, the initial workset and the initial solution set are identical. Also, the delta to the solution set
- * is consequently also the next workset.
- * 
- * Input files are plain text files and must be formatted as follows:
  *
- *   - Vertices represented as IDs and separated by new-line characters.
- *    
- * For example 
- *  {{{
- *  "1\n2\n12\n42\n63\n"
- *  }}}
- *  gives five vertices (1), (2), (12), (42), and (63).
- *  
- *  
- *   - Edges are represented as pairs for vertex IDs which are separated by space 
- * characters. Edges are separated by new-line characters.
- * 
- * For example 
- *  {{{
- *  "1 2\n2 12\n1 12\n42 63\n"
- *   }}}
- *   gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+ * Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the
+ * minimum of its own ID and its neighbors' IDs, as its new ID and tells its neighbors about its
+ * new ID. After the algorithm has completed, all vertices in the same component will have the same
+ * ID.
  *
+ * A vertex whose component ID did not change needs not propagate its information in the next
+ * step. Because of that, the algorithm is easily expressible via a delta iteration. We here model
+ * the solution set as the vertices with their current component ids, and the workset as the changed
+ * vertices. Because we see all vertices initially as changed, the initial workset and the initial
+ * solution set are identical. Also, the delta to the solution set is consequently also the next
+ * workset.
  * 
+ * Input files are plain text files and must be formatted as follows:
+ *
+ *   - Vertices represented as IDs and separated by new-line characters. For example,
+ *     `"1\n2\n12\n42\n63\n"` gives five vertices (1), (2), (12), (42), and (63).
+ *   - Edges are represented as pairs for vertex IDs which are separated by space characters. Edges
+ *     are separated by new-line characters. For example `"1 2\n2 12\n1 12\n42 63\n"`
+ *     gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
  *
  * Usage:
- *  {{{ 
- * 		ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>
- *   }}}
+ * {{{
+ *   ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>
+ * }}}
  *   
  * If no parameters are provided, the program is run with default data from
- *  [[org.apache.flink.example.java.graph.util.ConnectedComponentsData]]
- *  and 10 iterations. 
+ * [[org.apache.flink.example.java.graph.util.ConnectedComponentsData]] and 10 iterations.
  * 
  *
  * This example shows how to use:
@@ -78,46 +63,37 @@ object ConnectedComponents {
     if (!parseParameters(args)) {
       return
     }
-
     // set up execution environment
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    // read vertex and edge data 
+    // read vertex and edge data
     // assign the initial components (equal to the vertex id)
-    val vertices = getVerticesDataSet(env).map(id => (id, id))
-    
+    val vertices = getVerticesDataSet(env).map { id => (id, id) }
+
     // undirected edges by emitting for each input edge the input edges itself and an inverted version
-    val edges = getEdgesDataSet(env)
-      .flatMap { (edge, out: Collector[(Long, Long)]) => 
-      out.collect(edge)
-      out.collect((edge._2, edge._1))
-      }
-    
+    val edges = getEdgesDataSet(env).flatMap { edge => Seq(edge, (edge._2, edge._1)) }
+
     // open a delta iteration
     val verticesWithComponents = vertices.iterateDelta(vertices, maxIterations, Array(0)) {
-      (s, ws) => {
+      (s, ws) =>
+
+        // apply the step logic: join with the edges
+        val allNeighbors = ws.join(edges).where(0).equalTo(0) { (vertex, edge) =>
+          Some((edge._2, vertex._2))
+        }
+
+        // select the minimum neighbor
+        val minNeighbors = allNeighbors.groupBy(0).min(1)
 
-        // apply the step logic: join with the edges 
-        val allNeighbors = ws.join(edges)
-          .where(0).equalTo(0)
-          .map { in => (in._2._2, in._1._2) }
-        
-        // select the minimum neighbor 
-        val minNeighbors = allNeighbors.groupBy(0).aggregate(Aggregations.MIN, 1)
-        
         // update if the component of the candidate is smaller
-        val updatedComponents = minNeighbors.join(s).where(0).equalTo(0)
-        		.flatMap  { newAndOldComponent => 
-        		  newAndOldComponent match {
-        		    case ((vId, cNew), (_, cOld)) if cNew < cOld => Some((vId, cNew))
-        		    case _ => None 
-        		    }
-        		  }
+        val updatedComponents = minNeighbors.join(s).where(0).equalTo(0) {
+          (newVertex, oldVertex) => if (newVertex._2 < oldVertex._2) Some(newVertex) else None
+        }
+
         // delta and new workset are identical
-       (updatedComponents, updatedComponents)
-      }
+        (updatedComponents, updatedComponents)
     }
-      if (fileOutput) {
+    if (fileOutput) {
       verticesWithComponents.writeAsCsv(outputPath, "\n", " ")
     } else {
       verticesWithComponents.print()
@@ -154,10 +130,7 @@ object ConnectedComponents {
         .map { x => x._1 }
     }
     else {
-      val vertexData = ConnectedComponentsData.VERTICES map {
-        case Array(x) => x.asInstanceOf[Long]
-      }
-      env.fromCollection(vertexData);      
+      env.fromCollection(ConnectedComponentsData.VERTICES)
     }
   }
   
@@ -173,7 +146,7 @@ object ConnectedComponents {
       val edgeData = ConnectedComponentsData.EDGES map {
         case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
       }
-      env.fromCollection(edgeData);
+      env.fromCollection(edgeData)
     }
   }
 
@@ -182,4 +155,4 @@ object ConnectedComponents {
   private var edgesPath: String = null
   private var maxIterations: Int = 10
   private var outputPath: String = null
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
index 0bf01b4..0ad2e81 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
@@ -22,7 +22,7 @@ import scala.collection.JavaConverters._
 import org.apache.flink.api.scala.ExecutionEnvironment
 import org.apache.flink.api.common.functions.GroupReduceFunction
 import org.apache.flink.util.Collector
-import org.apache.flink.example.java.graph.util.EnumTrianglesData
+import org.apache.flink.examples.java.graph.util.EnumTrianglesData
 import org.apache.flink.api.common.operators.Order
 import scala.collection.mutable.MutableList
 
@@ -175,4 +175,4 @@ object EnumTrianglesBasic {
 	private var edgePath: String = null
 	private var outputPath: String = null
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
index e198986..24b7978 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
@@ -22,7 +22,7 @@ import scala.collection.JavaConverters._
 import org.apache.flink.api.scala.ExecutionEnvironment
 import org.apache.flink.api.common.functions.GroupReduceFunction
 import org.apache.flink.util.Collector
-import org.apache.flink.example.java.graph.util.EnumTrianglesData
+import org.apache.flink.examples.java.graph.util.EnumTrianglesData
 import org.apache.flink.api.common.operators.Order
 import scala.collection.mutable.MutableList
 
@@ -241,4 +241,4 @@ object EnumTrianglesOpt {
   private var edgePath: String = null
   private var outputPath: String = null
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
index 28a0e48..cdd04f7 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.examples.scala.graph
 
 import org.apache.flink.api.scala._
-import org.apache.flink.example.java.graph.util.PageRankData
+import org.apache.flink.examples.java.graph.util.PageRankData
 import org.apache.flink.api.java.aggregation.Aggregations.SUM
 
 import org.apache.flink.util.Collector
@@ -197,4 +197,4 @@ object PageRankBasic {
   private var numPages: Long = 0
   private var maxIterations: Int = 10
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
index 86d83db..4c79a1b 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
@@ -18,12 +18,11 @@
 package org.apache.flink.examples.scala.graph
 
 import org.apache.flink.api.scala._
-import org.apache.flink.example.java.graph.util.ConnectedComponentsData
+import org.apache.flink.examples.java.graph.util.ConnectedComponentsData
 import org.apache.flink.util.Collector
 
 object TransitiveClosureNaive {
 
-
 	def main (args: Array[String]): Unit = {
 		if (!parseParameters(args)) {
 			return
@@ -33,22 +32,22 @@ object TransitiveClosureNaive {
 
 		val edges = getEdgesDataSet(env)
 
-		val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long,Long)] =>
+		val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long, Long)] =>
 
 			val nextPaths = prevPaths
 				.join(edges)
 				.where(1).equalTo(0) {
-					(left,right) => Some((left._1,right._2))
+					(left, right) => Some((left._1,right._2))
 				}
 				.union(prevPaths)
-				.groupBy(0,1)
-				.reduce((l,r) => l)
+				.groupBy(0, 1)
+				.reduce((l, r) => l)
 
 			val terminate = prevPaths
 				.coGroup(nextPaths)
 				.where(0).equalTo(0) {
 					(prev, next, out: Collector[(Long, Long)]) => {
-						val prevPaths = prev.toList
+						val prevPaths = prev.toSet
 						for (n <- next)
 							if (!prevPaths.contains(n))
 								out.collect(n)
@@ -92,7 +91,7 @@ object TransitiveClosureNaive {
 			System.out.println("  See the documentation for the correct format of input files.")
 			System.out.println("  Usage: TransitiveClosure <edges path> <result path> <max number of iterations>")
 		}
-		return true
+		true
 	}
 
 	private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = {
@@ -111,96 +110,3 @@ object TransitiveClosureNaive {
 		}
 	}
 }
-
-
-
-
-
-
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.graph;
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//class TransitiveClosureNaive extends Program with ProgramDescription with Serializable {
-//
-//  def getScalaPlan(numSubTasks: Int, numIterations: Int, verticesInput: String, edgesInput: String, pathsOutput: String) = {
-//    val vertices = DataSource(verticesInput, DelimitedInputFormat(parseVertex))
-//    val edges = DataSource(edgesInput, DelimitedInputFormat(parseEdge))
-//
-//    def createClosure(paths: DataSetOLD[Path]) = {
-//
-//      val allNewPaths = paths join edges where { p => p.to } isEqualTo { e => e.from } map joinPaths
-//      val shortestPaths = allNewPaths union paths groupBy { p => (p.from, p.to) } reduceGroup { _ minBy { _.dist } }
-//
-//      shortestPaths
-//    }
-//
-//    val transitiveClosure = vertices.iterate(numIterations, createClosure)
-//
-//    val output = transitiveClosure.write(pathsOutput, DelimitedOutputFormat(formatOutput))
-//
-//    val plan = new ScalaPlan(Seq(output), "Transitive Closure (Naive)")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//
-//  def joinPaths = (p1: Path, p2: Path) => (p1, p2) match {
-//      case (Path(from, _, dist1), Path(_, to, dist2)) => Path(from, to, dist1 + dist2)
-//  }
-//
-//  case class Path(from: Int, to: Int, dist: Int)
-//
-//  def parseVertex = (line: String) => { val v = line.toInt; Path(v, v, 0) }
-//
-//  val EdgeInputPattern = """(\d+)\|(\d+)""".r
-//
-//  def parseEdge = (line: String) => line match {
-//    case EdgeInputPattern(from, to) => Path(from.toInt, to.toInt, 1)
-//  }
-//
-//  def formatOutput = (path: Path) => "%d|%d|%d".format(path.from, path.to, path.dist)
-//
-//  override def getDescription() = {
-//    "Parameters: <numSubStasks> <numIterations> <vertices> <edges> <output>"
-//  }
-//
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1).toInt, args(2), args(3), args(4))
-//  }
-//}
-//
-//object RunTransitiveClosureNaive {
-//  def main(pArgs: Array[String]) {
-//    if (pArgs.size < 3) {
-//      println("usage: [-numIterations <int:2>] -vertices <file> -edges <file> -output <file>")
-//      return
-//    }
-//    val args = Args.parse(pArgs)
-//    val plan = new TransitiveClosureNaive().getScalaPlan(2, args("numIterations", "10").toInt, args("vertices"), args("edges"), args("output"))
-//    LocalExecutor.execute(plan)
-//  }
-//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
index cf00d74..3e66275 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
@@ -23,8 +23,7 @@ import java.io.Serializable
 import org.apache.flink.api.common.functions._
 import org.apache.flink.api.scala._
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.example.java.clustering.util.KMeansData
-import org.apache.flink.example.java.ml.util.LinearRegressionData
+import org.apache.flink.examples.java.ml.util.LinearRegressionData
 
 import scala.collection.JavaConverters._
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
index f84b54c..156d635 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
@@ -18,14 +18,14 @@
 package org.apache.flink.examples.scala.relational
 
 import org.apache.flink.api.scala._
-import org.apache.flink.example.java.relational.util.WebLogData
+import org.apache.flink.examples.java.relational.util.WebLogData
 import org.apache.flink.util.Collector
 
 /**
  * This program processes web logs and relational data.
  * It implements the following relational query:
  *
- * <code><pre>
+ * {{{
  * SELECT
  *       r.pageURL,
  *       r.pageRank,
@@ -40,13 +40,15 @@ import org.apache.flink.util.Collector
  *              WHERE v.destUrl = d.url
  *                    AND v.visitDate < [date]
  *           );
- * </pre></code>
+ * }}}
+ *
  *
- * <p>
  * Input files are plain text CSV files using the pipe character ('|') as field separator.
- * The tables referenced in the query can be generated using the {@link WebLogDataGenerator} and
+ * The tables referenced in the query can be generated using the
+ * [org.apache.flink.example.java.relational.util.WebLogDataGenerator]] and
  * have the following schemas
- * <code><pre>
+ *
+ * {{{
  * CREATE TABLE Documents (
  *                url VARCHAR(100) PRIMARY KEY,
  *                contents TEXT );
@@ -66,133 +68,143 @@ import org.apache.flink.util.Collector
  *                languageCode VARCHAR(6),
  *                searchWord VARCHAR(32),
  *                duration INT );
- * </pre></code>
+ * }}}
+ *
  *
- * <p>
- * Usage: <code>WebLogAnalysis &lt;documents path&gt; &lt;ranks path&gt; &lt;visits path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WebLogData}.
+ * Usage
+ * {{{
+ *   WebLogAnalysis <documents path> <ranks path> <visits path> <result path>
+ * }}}
+ *
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.example.java.relational.util.WebLogData]].
  *
- * <p>
  * This example shows how to use:
- * <ul>
- * <li> tuple data types
- * <li> projection and join projection
- * <li> the CoGroup transformation for an anti-join
- * </ul>
+ *
+ *  - tuple data types
+ *  - projection and join projection
+ *  - the CoGroup transformation for an anti-join
  *
  */
 object WebLogAnalysis {
 
-	def main(args: Array[String]) {
-		if (!parseParameters(args)) {
-			return
-		}
-
-		val env = ExecutionEnvironment.getExecutionEnvironment
-
-		val documents = getDocumentsDataSet(env)
-		val ranks = getRanksDataSet(env)
-		val visits = getVisitsDataSet(env)
-
-		val filteredDocs = documents
-			.filter (doc => doc._2.contains(" editors ") && doc._2.contains ( " oscillations "))
-
-		val filteredRanks = ranks
-			.filter (rank => rank._1 > 40)
-
-		val filteredVisits = visits
-			.filter (visit => visit._2.substring(0,4).toInt == 2007)
-
-		val joinDocsRanks = filteredDocs
-			.join(filteredRanks)
-			.where(0).equalTo(1)
-			.map(_._2)
-
-		val result = joinDocsRanks
-			.coGroup(filteredVisits)
-			.where(1).equalTo(0)
-			.apply ((ranks, visits, out:Collector[(Int,String,Int)]) => {
-				if (visits.isEmpty) for (rank <- ranks) out.collect(rank)
-		})
-
-		result.print()
-
-		env.execute("WebLogAnalysis Example")
-	}
-
-
-	private var fileOutput: Boolean = false
-	private var documentsPath: String = null
-	private var ranksPath: String = null
-	private var visitsPath: String = null
-	private var outputPath: String = null
-
-	private def parseParameters (args: Array[String]) : Boolean = {
-		if (args.length > 0) {
-			fileOutput = true;
-			if (args.length == 4) {
-				documentsPath = args(0)
-				ranksPath = args(1)
-				visitsPath = args(2)
-				outputPath = args(3)
-			}
-			else {
-				System.err.println("Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>")
-				return false
-			}
-		}
-		else {
-			System.out.println("Executing WebLog Analysis example with built-in default data.")
-			System.out.println("  Provide parameters to read input data from files.")
-			System.out.println("  See the documentation for the correct format of input files.")
-			System.out.println("  We provide a data generator to create synthetic input files for this program.")
-			System.out.println("  Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>")
-		}
-		return true;
-	}
-
-	private def getDocumentsDataSet(env: ExecutionEnvironment): DataSet[(String,String)] = {
-		if (fileOutput) {
-			env.readCsvFile[(String,String)](
-				documentsPath,
-				fieldDelimiter = '|',
-				includedFields = Array(0,1))
-		}
-		else {
-			val documents = WebLogData.DOCUMENTS map {
-				case Array(x,y) => (x.asInstanceOf[String],y.asInstanceOf[String])
-			}
-			env.fromCollection(documents)
-		}
-	}
-
-	private def getRanksDataSet(env: ExecutionEnvironment) : DataSet[(Int, String, Int)] = {
-		if (fileOutput) {
-			env.readCsvFile[(Int,String,Int)](
-				ranksPath,
-				fieldDelimiter = '|',
-				includedFields = Array(0,1,2))
-		}
-		else {
-			val ranks = WebLogData.RANKS map {
-				case Array(x,y,z) => (x.asInstanceOf[Int], y.asInstanceOf[String], z.asInstanceOf[Int])
-			}
-			env.fromCollection(ranks)
-		}
-	}
-
-	private def getVisitsDataSet (env: ExecutionEnvironment) : DataSet[(String,String)] = {
-		if (fileOutput) {
-			env.readCsvFile[(String,String)](
-				visitsPath,
-				fieldDelimiter = '|',
-				includedFields = Array(0,1))
-		}
-		else {
-			val visits = WebLogData.VISITS map {
-				case Array(x,y) => (x.asInstanceOf[String], y.asInstanceOf[String])
-			}
-			env.fromCollection(visits)
-		}
-	}
-}
\ No newline at end of file
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    val documents = getDocumentsDataSet(env)
+    val ranks = getRanksDataSet(env)
+    val visits = getVisitsDataSet(env)
+
+    val filteredDocs = documents
+      .filter(doc => doc._2.contains(" editors ") && doc._2.contains(" oscillations "))
+
+    val filteredRanks = ranks
+      .filter(rank => rank._1 > 40)
+
+    val filteredVisits = visits
+      .filter(visit => visit._2.substring(0, 4).toInt == 2007)
+
+    val joinDocsRanks = filteredDocs.join(filteredRanks).where(0).equalTo(1) {
+      (doc, rank) => Some(rank)
+    }
+
+    val result = joinDocsRanks.coGroup(filteredVisits).where(1).equalTo(0) {
+      (ranks, visits, out: Collector[(Int, String, Int)]) =>
+        if (visits.isEmpty) for (rank <- ranks) out.collect(rank)
+    }
+
+
+
+
+    // emit result
+    if (fileOutput) {
+      result.writeAsCsv(outputPath, "\n", "|")
+    } else {
+      result.print()
+    }
+
+    env.execute("Scala WebLogAnalysis Example")
+  }
+
+  private var fileOutput: Boolean = false
+  private var documentsPath: String = null
+  private var ranksPath: String = null
+  private var visitsPath: String = null
+  private var outputPath: String = null
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      fileOutput = true
+      if (args.length == 4) {
+        documentsPath = args(0)
+        ranksPath = args(1)
+        visitsPath = args(2)
+        outputPath = args(3)
+      }
+      else {
+        System.err.println("Usage: WebLogAnalysis <documents path> <ranks path> <visits path> " +
+          "<result path>")
+        return false
+      }
+    }
+    else {
+      System.out.println("Executing WebLog Analysis example with built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  We provide a data generator to create synthetic input files for this " +
+        "program.")
+      System.out.println("  Usage: WebLogAnalysis <documents path> <ranks path> <visits path> " +
+        "<result path>")
+    }
+    true
+  }
+
+  private def getDocumentsDataSet(env: ExecutionEnvironment): DataSet[(String, String)] = {
+    if (fileOutput) {
+      env.readCsvFile[(String, String)](
+        documentsPath,
+        fieldDelimiter = '|',
+        includedFields = Array(0, 1))
+    }
+    else {
+      val documents = WebLogData.DOCUMENTS map {
+        case Array(x, y) => (x.asInstanceOf[String], y.asInstanceOf[String])
+      }
+      env.fromCollection(documents)
+    }
+  }
+
+  private def getRanksDataSet(env: ExecutionEnvironment): DataSet[(Int, String, Int)] = {
+    if (fileOutput) {
+      env.readCsvFile[(Int, String, Int)](
+        ranksPath,
+        fieldDelimiter = '|',
+        includedFields = Array(0, 1, 2))
+    }
+    else {
+      val ranks = WebLogData.RANKS map {
+        case Array(x, y, z) => (x.asInstanceOf[Int], y.asInstanceOf[String], z.asInstanceOf[Int])
+      }
+      env.fromCollection(ranks)
+    }
+  }
+
+  private def getVisitsDataSet(env: ExecutionEnvironment): DataSet[(String, String)] = {
+    if (fileOutput) {
+      env.readCsvFile[(String, String)](
+        visitsPath,
+        fieldDelimiter = '|',
+        includedFields = Array(1, 2))
+    }
+    else {
+      val visits = WebLogData.VISITS map {
+        case Array(x, y) => (x.asInstanceOf[String], y.asInstanceOf[String])
+      }
+      env.fromCollection(visits)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
index 22838e5..9d93ca8 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.examples.scala.wordcount
 
 import org.apache.flink.api.scala._
-import org.apache.flink.example.java.wordcount.util.WordCountData
+import org.apache.flink.examples.java.wordcount.util.WordCountData
 
 /**
  * Implements the "WordCount" program that computes a simple word occurrence histogram

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
index eb6fe9f..fb49237 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
@@ -30,11 +30,11 @@ import org.apache.flink.compiler.plan.BulkPartialSolutionPlanNode;
 import org.apache.flink.compiler.plan.OptimizedPlan;
 import org.apache.flink.compiler.plan.SinkPlanNode;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.example.java.graph.PageRankBasic.BuildOutgoingEdgeList;
-import org.apache.flink.example.java.graph.PageRankBasic.Dampener;
-import org.apache.flink.example.java.graph.PageRankBasic.EpsilonFilter;
-import org.apache.flink.example.java.graph.PageRankBasic.JoinVertexWithEdgesMatch;
-import org.apache.flink.example.java.graph.PageRankBasic.RankAssigner;
+import org.apache.flink.examples.java.graph.PageRankBasic.BuildOutgoingEdgeList;
+import org.apache.flink.examples.java.graph.PageRankBasic.Dampener;
+import org.apache.flink.examples.java.graph.PageRankBasic.EpsilonFilter;
+import org.apache.flink.examples.java.graph.PageRankBasic.JoinVertexWithEdgesMatch;
+import org.apache.flink.examples.java.graph.PageRankBasic.RankAssigner;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
 import org.apache.flink.test.compiler.util.CompilerTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
index 11102d7..6f17197 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
@@ -23,7 +23,7 @@ import org.apache.flink.client.program.Client.ProgramAbortException;
 import org.apache.flink.client.program.PackagedProgram.PreviewPlanEnvironment;
 import org.apache.flink.compiler.plan.OptimizedPlan;
 import org.apache.flink.compiler.plandump.PlanJSONDumpGenerator;
-import org.apache.flink.example.java.clustering.KMeans;
+import org.apache.flink.examples.java.clustering.KMeans;
 import org.apache.flink.test.compiler.util.CompilerTestBase;
 import org.apache.flink.test.recordJobs.graph.DeltaPageRankWithInitialDeltas;
 import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java
index 38bf135..3c9be4b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java
@@ -21,7 +21,7 @@ package org.apache.flink.test.exampleJavaPrograms;
 
 import java.io.BufferedReader;
 
-import org.apache.flink.example.java.graph.ConnectedComponents;
+import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
index 790bd90..bc627c3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.test.exampleJavaPrograms;
 
-import org.apache.flink.example.java.graph.EnumTrianglesBasic;
+import org.apache.flink.examples.java.graph.EnumTrianglesBasic;
 import org.apache.flink.test.testdata.EnumTriangleData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleOptITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleOptITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleOptITCase.java
index 7cd6301..5d20362 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleOptITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleOptITCase.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.test.exampleJavaPrograms;
 
-import org.apache.flink.example.java.graph.EnumTrianglesOpt;
+import org.apache.flink.examples.java.graph.EnumTrianglesOpt;
 import org.apache.flink.test.testdata.EnumTriangleData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
index 253ff17..62a0347 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
@@ -24,7 +24,7 @@ import java.util.Collection;
 import java.util.LinkedList;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.example.java.graph.PageRankBasic;
+import org.apache.flink.examples.java.graph.PageRankBasic;
 import org.apache.flink.test.testdata.PageRankData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java
index ee79447..183b4fb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java
@@ -22,7 +22,7 @@ package org.apache.flink.test.exampleJavaPrograms;
 
 import java.io.BufferedReader;
 
-import org.apache.flink.example.java.graph.TransitiveClosureNaive;
+import org.apache.flink.examples.java.graph.TransitiveClosureNaive;
 import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.testdata.TransitiveClosureData;
 import org.apache.flink.test.util.JavaProgramTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
index 322bbb9..5e5c212 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
@@ -19,7 +19,7 @@
 package org.apache.flink.test.exampleJavaPrograms;
 
 
-import org.apache.flink.example.java.relational.WebLogAnalysis;
+import org.apache.flink.examples.java.relational.WebLogAnalysis;
 import org.apache.flink.test.testdata.WebLogAnalysisData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
index 199ba34..02febea 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.test.exampleJavaPrograms;
 
-import org.apache.flink.example.java.wordcount.WordCount;
+import org.apache.flink.examples.java.wordcount.WordCount;
 import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java
index 0fe24c5..df083f3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.api.java.aggregation.Aggregations;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.example.java.wordcount.WordCount;
+import org.apache.flink.examples.java.wordcount.WordCount;
 import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
index 06a8eaf..421d823 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
@@ -34,8 +34,8 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.example.java.graph.ConnectedComponents.DuplicateValue;
-import org.apache.flink.example.java.graph.ConnectedComponents.NeighborWithComponentIDJoin;
+import org.apache.flink.examples.java.graph.ConnectedComponents.DuplicateValue;
+import org.apache.flink.examples.java.graph.ConnectedComponents.NeighborWithComponentIDJoin;
 
 @SuppressWarnings("serial")
 public class CoGroupConnectedComponentsSecondITCase extends JavaProgramTestBase {


[13/60] Move RichFunctions to api.common package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
index b94840f..edb1c74 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
@@ -64,15 +64,15 @@ import org.apache.flink.api.common.InvalidProgramException;
  * </b>
  * <p>
  * Be aware that some annotations should only be used for functions with as single input
- * ({@link RichMapFunction}, {@link RichReduceFunction}) and some only for stubs with two inputs
- * ({@link RichCrossFunction}, {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}).
+ * ({@link org.apache.flink.api.common.functions.RichMapFunction}, {@link org.apache.flink.api.common.functions.RichReduceFunction}) and some only for stubs with two inputs
+ * ({@link org.apache.flink.api.common.functions.RichCrossFunction}, {@link org.apache.flink.api.common.functions.RichFlatJoinFunction}, {@link org.apache.flink.api.common.functions.RichCoGroupFunction}).
  */
 public class FunctionAnnotation {
 
 	/**
 	 * This annotation declares that a function leaves certain fields of its input values unmodified and
 	 * only "forwards" or "copies" them to the return value. The annotation is applicable to unary
-	 * functions, like for example {@link RichMapFunction}, {@link RichReduceFunction}, or {@link RichFlatMapFunction}.
+	 * functions, like for example {@link org.apache.flink.api.common.functions.RichMapFunction}, {@link org.apache.flink.api.common.functions.RichReduceFunction}, or {@link org.apache.flink.api.common.functions.RichFlatMapFunction}.
 	 * <p>
 	 * The following example illustrates a function that keeps the tuple's field zero constant:
 	 * <pre><blockquote>
@@ -103,7 +103,7 @@ public class FunctionAnnotation {
 	/**
 	 * This annotation declares that a function leaves certain fields of its first input values unmodified and
 	 * only "forwards" or "copies" them to the return value. The annotation is applicable to binary
-	 * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}.
+	 * functions, like for example {@link org.apache.flink.api.common.functions.RichFlatJoinFunction}, {@link org.apache.flink.api.common.functions.RichCoGroupFunction}, or {@link org.apache.flink.api.common.functions.RichCrossFunction}.
 	 * <p>
 	 * The following example illustrates a join function that copies fields from the first and second input to the
 	 * return value:
@@ -135,7 +135,7 @@ public class FunctionAnnotation {
 	/**
 	 * This annotation declares that a function leaves certain fields of its second input values unmodified and
 	 * only "forwards" or "copies" them to the return value. The annotation is applicable to binary
-	 * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}.
+	 * functions, like for example {@link org.apache.flink.api.common.functions.RichFlatJoinFunction}, {@link org.apache.flink.api.common.functions.RichCoGroupFunction}, or {@link org.apache.flink.api.common.functions.RichCrossFunction}.
 	 * <p>
 	 * The following example illustrates a join function that copies fields from the first and second input to the
 	 * return value:
@@ -167,7 +167,7 @@ public class FunctionAnnotation {
 	/**
 	 * This annotation declares that a function changes certain fields of its input values, while leaving all
 	 * others unmodified and in place in the return value. The annotation is applicable to unary
-	 * functions, like for example {@link RichMapFunction}, {@link RichReduceFunction}, or {@link RichFlatMapFunction}.
+	 * functions, like for example {@link org.apache.flink.api.common.functions.RichMapFunction}, {@link org.apache.flink.api.common.functions.RichReduceFunction}, or {@link org.apache.flink.api.common.functions.RichFlatMapFunction}.
 	 * <p>
 	 * The following example illustrates that at the example of a Map function:
 	 * 
@@ -201,7 +201,7 @@ public class FunctionAnnotation {
 	/**
 	 * This annotation declares that a function changes certain fields of its first input value, while leaving all
 	 * others unmodified and in place in the return value. The annotation is applicable to binary
-	 * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}.
+	 * functions, like for example {@link org.apache.flink.api.common.functions.RichFlatJoinFunction}, {@link org.apache.flink.api.common.functions.RichCoGroupFunction}, or {@link org.apache.flink.api.common.functions.RichCrossFunction}.
 	 * <p>
 	 * The following example illustrates a join function that copies fields from the first and second input to the
 	 * return value:
@@ -238,7 +238,7 @@ public class FunctionAnnotation {
 	/**
 	 * This annotation declares that a function changes certain fields of its second input value, while leaving all
 	 * others unmodified and in place in the return value. The annotation is applicable to binary
-	 * functions, like for example {@link RichFlatJoinFunction}, {@link RichCoGroupFunction}, or {@link RichCrossFunction}.
+	 * functions, like for example {@link org.apache.flink.api.common.functions.RichFlatJoinFunction}, {@link org.apache.flink.api.common.functions.RichCoGroupFunction}, or {@link org.apache.flink.api.common.functions.RichCrossFunction}.
 	 * <p>
 	 * The following example illustrates a join function that copies fields from the first and second input to the
 	 * return value:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java
index 087808d..2363e4f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/GroupReduceIterator.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.functions;
 
 import java.util.Iterator;
 
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.util.Collector;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java
deleted file mode 100644
index 3169622..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCoGroupFunction.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Rich variant of the {@link CoGroupFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <IN1> The type of the elements in the first input.
- * @param <IN2> The type of the elements in the second input.
- * @param <OUT> The type of the result elements.
- */
-public abstract class RichCoGroupFunction<IN1, IN2, OUT> extends AbstractRichFunction implements CoGroupFunction<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-	
-	@Override
-	public abstract void coGroup(Iterable<IN1> first, Iterable<IN2> second, Collector<OUT> out) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java
deleted file mode 100644
index 58be279..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichCrossFunction.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * Rich variant of the {@link CrossFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <IN1> The type of the elements in the first input.
- * @param <IN2> The type of the elements in the second input.
- * @param <OUT> The type of the result elements.
- */
-public abstract class RichCrossFunction<IN1, IN2, OUT> extends AbstractRichFunction implements CrossFunction<IN1, IN2, OUT> {
-	
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract OUT cross(IN1 first, IN2 second) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java
deleted file mode 100644
index 9057a0f..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFilterFunction.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * Rich variant of the {@link FilterFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <T> The type of the filtered elements.
- */
-public abstract class RichFilterFunction<T> extends AbstractRichFunction implements FilterFunction<T> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	@Override
-	public abstract boolean filter(T value) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java
deleted file mode 100644
index a5d45aa..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatCombineFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FlatCombineFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Rich variant of the {@link FlatCombineFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- *
- * @param <T> The data type of the elements to be combined.
- */
-public abstract class RichFlatCombineFunction<T> extends AbstractRichFunction implements FlatCombineFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract void combine(Iterable<T> values, Collector<T> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java
deleted file mode 100644
index 6918364..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatJoinFunction.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Rich variant of the {@link FlatJoinFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <IN1> The type of the elements in the first input.
- * @param <IN2> The type of the elements in the second input.
- * @param <OUT> The type of the result elements.
- */
-public abstract class RichFlatJoinFunction<IN1, IN2, OUT> extends AbstractRichFunction implements FlatJoinFunction<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract void join(IN1 first, IN2 second, Collector<OUT> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java
deleted file mode 100644
index c045508..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichFlatMapFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Rich variant of the {@link FlatMapFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <IN> Type of the input elements.
- * @param <OUT> Type of the returned elements.
- */
-public abstract class RichFlatMapFunction<IN, OUT> extends AbstractRichFunction implements FlatMapFunction<IN, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract void flatMap(IN value, Collector<OUT> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java
deleted file mode 100644
index 9198aeb..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichGroupReduceFunction.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FlatCombineFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Rich variant of the {@link GroupReduceFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <IN> Type of the elements that this function processes.
- * @param <OUT> The type of the elements returned by the user-defined function.
- */
-public abstract class RichGroupReduceFunction<IN, OUT> extends AbstractRichFunction implements GroupReduceFunction<IN, OUT>, FlatCombineFunction<IN> {
-	
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract void reduce(Iterable<IN> values, Collector<OUT> out) throws Exception;
-	
-	/**
-	 * The combine methods pre-reduces elements. It may be called on subsets of the data
-	 * before the actual reduce function. This is often helpful to lower data volume prior
-	 * to reorganizing the data in an expensive way, as might be required for the final
-	 * reduce function.
-	 * <p>
-	 * This method is only ever invoked when the subclass of {@link RichGroupReduceFunction}
-	 * adds the {@link Combinable} annotation, or if the <i>combinable</i> flag is set when defining
-	 * the <i>reduceGroup<i> operation via
-	 * {@link org.apache.flink.api.java.operators.GroupReduceOperator#setCombinable(boolean)}.
-	 * <p>
-	 * Since the reduce function will be called on the result of this method, it is important that this
-	 * method returns the same data type as it consumes. By default, this method only calls the
-	 * {@link #reduce(Iterable, Collector)} method. If the behavior in the pre-reducing is different
-	 * from the final reduce function (for example because the reduce function changes the data type),
-	 * this method must be overwritten, or the execution will fail.
-	 * 
-	 * @param values The iterator returning the group of values to be reduced.
-	 * @param out The collector to emit the returned values.
-	 * 
-	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
-	 *                   to fail and may trigger recovery.
-	 */
-	@Override
-	public void combine(Iterable<IN> values, Collector<IN> out) throws Exception {
-		@SuppressWarnings("unchecked")
-		Collector<OUT> c = (Collector<OUT>) out;
-		reduce(values, c);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * This annotation can be added to classes that extend {@link RichGroupReduceFunction}, in oder to mark
-	 * them as "combinable". The system may call the {@link RichGroupReduceFunction#combine(Iterable, Collector)}
-	 * method on such functions, to pre-reduce the data before transferring it over the network to
-	 * the actual group reduce operation.
-	 * <p>
-	 * Marking combinable functions as such is in general beneficial for performance.
-	 */
-	@Retention(RetentionPolicy.RUNTIME)
-	@Target(ElementType.TYPE)
-	public static @interface Combinable {};
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java
deleted file mode 100644
index a0c28ee..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichJoinFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * Rich variant of the {@link JoinFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- *
- * @param <IN1> The type of the elements in the first input.
- * @param <IN2> The type of the elements in the second input.
- * @param <OUT> The type of the result elements.
- */
-public abstract class RichJoinFunction<IN1,IN2,OUT> extends AbstractRichFunction implements JoinFunction<IN1,IN2,OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract OUT join(IN1 first, IN2 second) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java
deleted file mode 100644
index f6f5356..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapFunction.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * Rich variant of the {@link MapFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <IN> Type of the input elements.
- * @param <OUT> Type of the returned elements.
- */
-public abstract class RichMapFunction<IN, OUT> extends AbstractRichFunction implements MapFunction<IN, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract OUT map(IN value) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapPartitionFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapPartitionFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapPartitionFunction.java
deleted file mode 100644
index 4a3beab..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichMapPartitionFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.MapPartitionFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * Rich variant of the {@link MapPartitionFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <I> Type of the input elements.
- * @param <O> Type of the returned elements.
- */
-public abstract class RichMapPartitionFunction<I, O> extends AbstractRichFunction implements MapPartitionFunction<I, O> {
-
-	private static final long serialVersionUID = 1L;
-	
-	@Override
-	public abstract void mapPartition(Iterable<I> values, Collector<O> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java
deleted file mode 100644
index a63f8dc..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/RichReduceFunction.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * Rich variant of the {@link ReduceFunction}. As a {@link RichFunction}, it gives access to the
- * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods:
- * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and
- * {@link RichFunction#close()}.
- * 
- * @param <T> Type of the elements that this function processes.
- */
-public abstract class RichReduceFunction<T> extends AbstractRichFunction implements ReduceFunction<T> {
-	
-	private static final long serialVersionUID = 1L;
-
-	public abstract T reduce(T value1, T value2) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
index d1fed5e..be5fe48 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
@@ -33,8 +33,8 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.aggregation.AggregationFunction;
 import org.apache.flink.api.java.aggregation.AggregationFunctionFactory;
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.configuration.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
index 81f29e2..ab61d04 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
@@ -452,7 +452,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 
 			/**
 			 * Intermediate step of a CoGroup transformation. <br/>
-			 * To continue the CoGroup transformation, provide a {@link org.apache.flink.api.java.functions.RichCoGroupFunction} by calling
+			 * To continue the CoGroup transformation, provide a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} by calling
 			 * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.common.functions.CoGroupFunction)}.
 			 *
 			 */
@@ -506,13 +506,13 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				}
 
 				/**
-				 * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.java.functions.RichCoGroupFunction} to groups of elements with identical keys.<br/>
+				 * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} to groups of elements with identical keys.<br/>
 				 * Each CoGroupFunction call returns an arbitrary number of keys. 
 				 * 
 				 * @param function The CoGroupFunction that is called for all groups of elements with identical keys.
 				 * @return An CoGroupOperator that represents the co-grouped result DataSet.
 				 * 
-				 * @see org.apache.flink.api.java.functions.RichCoGroupFunction
+				 * @see org.apache.flink.api.common.functions.RichCoGroupFunction
 				 * @see DataSet
 				 */
 				public <R> CoGroupOperator<I1, I2, R> with(CoGroupFunction<I1, I2, R> function) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
index 7d8a28f..7c063cc 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction.Combinable;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
 import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
index 9de8bd3..a8bdca7 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.operators.Ordering;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
 import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index 2875251..f242453 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -34,7 +34,7 @@ import org.apache.flink.api.common.operators.base.JoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder;
-import org.apache.flink.api.java.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.functions.SemanticPropUtil;
 import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
@@ -153,7 +153,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 	 * @param <I2> The type of the second input DataSet of the Join transformation.
 	 * @param <OUT> The type of the result of the Join transformation.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichFlatJoinFunction
+	 * @see org.apache.flink.api.common.functions.RichFlatJoinFunction
 	 * @see DataSet
 	 */
 	public static class EquiJoin<I1, I2, OUT> extends JoinOperator<I1, I2, OUT> {
@@ -487,13 +487,13 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 		
 		/**
-		 * Finalizes a Join transformation by applying a {@link org.apache.flink.api.java.functions.RichFlatJoinFunction} to each pair of joined elements.<br/>
+		 * Finalizes a Join transformation by applying a {@link org.apache.flink.api.common.functions.RichFlatJoinFunction} to each pair of joined elements.<br/>
 		 * Each JoinFunction call returns exactly one element. 
 		 * 
 		 * @param function The JoinFunction that is called for each pair of joined elements.
 		 * @return An EquiJoin that represents the joined result DataSet
 		 * 
-		 * @see org.apache.flink.api.java.functions.RichFlatJoinFunction
+		 * @see org.apache.flink.api.common.functions.RichFlatJoinFunction
 		 * @see org.apache.flink.api.java.operators.JoinOperator.EquiJoin
 		 * @see DataSet
 		 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
index dcdbed4..5f81b62 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
@@ -35,8 +35,8 @@ import org.apache.flink.api.java.DataSet;
 /**
  * The <tt>SingleInputUdfOperator</tt> is the base class of all unary operators that execute
  * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that
- * have one input (such as {@link org.apache.flink.api.java.functions.RichMapFunction} or
- * {@link org.apache.flink.api.java.functions.RichReduceFunction}).
+ * have one input (such as {@link org.apache.flink.api.common.functions.RichMapFunction} or
+ * {@link org.apache.flink.api.common.functions.RichReduceFunction}).
  * <p>
  * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization
  * through configuration objects, and semantic properties.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
index 1647055..ed252c4 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
@@ -70,14 +70,14 @@ public class SortedGrouping<T> extends Grouping<T> {
 
 	/**
 	 * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}.<br/>
-	 * The transformation calls a {@link org.apache.flink.api.java.functions.RichGroupReduceFunction} for each group of the DataSet.
+	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet.
 	 * A GroupReduceFunction can iterate over all elements of a group and emit any
 	 *   number of output elements including none.
 	 * 
 	 * @param reducer The GroupReduceFunction that is applied on each group of the DataSet.
 	 * @return A GroupReduceOperator that represents the reduced DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichGroupReduceFunction
+	 * @see org.apache.flink.api.common.functions.RichGroupReduceFunction
 	 * @see GroupReduceOperator
 	 * @see DataSet
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
index f347fef..030ad3f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
@@ -35,8 +35,8 @@ import org.apache.flink.api.java.DataSet;
 /**
  * The <tt>TwoInputUdfOperator</tt> is the base class of all binary operators that execute
  * user-defined functions (UDFs). The UDFs encapsulated by this operator are naturally UDFs that
- * have two inputs (such as {@link org.apache.flink.api.java.functions.RichJoinFunction} or
- * {@link org.apache.flink.api.java.functions.RichCoGroupFunction}).
+ * have two inputs (such as {@link org.apache.flink.api.common.functions.RichJoinFunction} or
+ * {@link org.apache.flink.api.common.functions.RichCoGroupFunction}).
  * <p>
  * This class encapsulates utilities for the UDFs, such as broadcast variables, parameterization
  * through configuration objects, and semantic properties.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
index bf33f4e..8af3322 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java
@@ -27,8 +27,8 @@ import org.apache.flink.api.java.DataSet;
 
 /**
  * This interface marks operators as operators that execute user-defined functions (UDFs), such as
- * {@link org.apache.flink.api.java.functions.RichMapFunction}, {@link org.apache.flink.api.java.functions.RichReduceFunction},
- * or {@link org.apache.flink.api.java.functions.RichCoGroupFunction}.
+ * {@link org.apache.flink.api.common.functions.RichMapFunction}, {@link org.apache.flink.api.common.functions.RichReduceFunction},
+ * or {@link org.apache.flink.api.common.functions.RichCoGroupFunction}.
  * The UDF operators stand in contrast to operators that execute built-in operations, like aggregations.
  */
 public interface UdfOperator<O extends UdfOperator<O>> {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
index fdc86de..54a841b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
@@ -97,14 +97,14 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	
 	/**
 	 * Applies a Reduce transformation on a grouped {@link DataSet}.<br/>
-	 * For each group, the transformation consecutively calls a {@link org.apache.flink.api.java.functions.RichReduceFunction}
+	 * For each group, the transformation consecutively calls a {@link org.apache.flink.api.common.functions.RichReduceFunction}
 	 *   until only a single element for each group remains. 
 	 * A ReduceFunction combines two elements into one new element of the same type.
 	 * 
 	 * @param reducer The ReduceFunction that is applied on each group of the DataSet.
 	 * @return A ReduceOperator that represents the reduced DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichReduceFunction
+	 * @see org.apache.flink.api.common.functions.RichReduceFunction
 	 * @see ReduceOperator
 	 * @see DataSet
 	 */
@@ -117,14 +117,14 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	
 	/**
 	 * Applies a GroupReduce transformation on a grouped {@link DataSet}.<br/>
-	 * The transformation calls a {@link org.apache.flink.api.java.functions.RichGroupReduceFunction} for each group of the DataSet.
+	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet.
 	 * A GroupReduceFunction can iterate over all elements of a group and emit any
 	 *   number of output elements including none.
 	 * 
 	 * @param reducer The GroupReduceFunction that is applied on each group of the DataSet.
 	 * @return A GroupReduceOperator that represents the reduced DataSet.
 	 * 
-	 * @see org.apache.flink.api.java.functions.RichGroupReduceFunction
+	 * @see org.apache.flink.api.common.functions.RichGroupReduceFunction
 	 * @see GroupReduceOperator
 	 * @see DataSet
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java
index c7f65f0..6803618 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyExtractingMapper.java
@@ -19,7 +19,7 @@
 package org.apache.flink.api.java.operators.translation;
 
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java
index a6cd837..8213d11 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/KeyRemovingMapper.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.api.java.operators.translation;
 
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java
index 7e3f0e5..c222ff2 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.FlatCombineFunction;
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.operators.Keys;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.types.TypeInformation;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java
index 474b022..c0f6ba6 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java
@@ -22,6 +22,8 @@ package org.apache.flink.api.java.functions;
 import static org.junit.Assert.*;
 
 import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.operators.DualInputSemanticProperties;
 import org.apache.flink.api.common.operators.SingleInputSemanticProperties;
 import org.apache.flink.api.common.operators.base.GenericDataSinkBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
index 71eb682..e49c54c 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
@@ -35,9 +35,9 @@ import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
-import org.apache.flink.api.java.functions.RichJoinFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.util.Collector;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
index 8e457ce..b770bad 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.operators.base.GenericDataSourceBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.common.operators.base.ReduceOperatorBase;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
index 412f751..f21045e 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
@@ -24,14 +24,14 @@ import java.io.IOException;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
-import org.apache.flink.api.java.functions.RichCrossFunction;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichCrossFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.InvalidTypesException;
-import org.apache.flink.api.java.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/LambdaExtractionTest.java
----------------------------------------------------------------------
diff --git a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/LambdaExtractionTest.java b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/LambdaExtractionTest.java
index 4c8ee23..fbb7ac8 100644
--- a/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/LambdaExtractionTest.java
+++ b/flink-java8-tests/src/test/java/org/apache/flink/test/javaApiOperators/lambdas/LambdaExtractionTest.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.*;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java
index 3c8bbd0..8cbcd58 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java
@@ -23,12 +23,9 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.java.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory;
-import org.apache.flink.runtime.operators.BuildFirstCachedMatchDriver;
-import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;
 import org.apache.flink.runtime.operators.testutils.ExpectedTestException;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java
index b65f161..e2cee91 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java
@@ -20,11 +20,9 @@ package org.apache.flink.runtime.operators;
 
 import org.junit.Assert;
 import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory;
-import org.apache.flink.runtime.operators.CoGroupDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;
 import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator;
 import org.apache.flink.types.IntValue;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java
index 968d947..e4fed9c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskTest.java
@@ -22,11 +22,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.junit.Assert;
 import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordPairComparatorFactory;
-import org.apache.flink.runtime.operators.CoGroupDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.CoGroupTaskExternalITCase.MockCoGroupStub;
 import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java
index 885a509..e08396e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskExternalITCase.java
@@ -24,7 +24,7 @@ import java.util.HashMap;
 
 import org.junit.Assert;
 
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.runtime.operators.CombineTaskTest.MockCombiningReduceStub;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java
index 7915d1f..c71e81c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java
@@ -23,11 +23,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.junit.Assert;
 
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.GroupReduceCombineDriver;
 import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java
index af58a17..47b0b03 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java
@@ -25,12 +25,10 @@ import org.junit.Assert;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.GroupReduceDriver;
 import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;
 import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java
index 97b8950..fb44671 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java
@@ -26,12 +26,10 @@ import org.junit.Assert;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.GroupReduceDriver;
 import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DelayingInfinitiveInputIterator;
 import org.apache.flink.runtime.operators.testutils.DriverTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
index 070097c..e7ca6f3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.apache.flink.api.common.functions.GenericCollectorMap;
 import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparatorFactory;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory;
 import org.apache.flink.configuration.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java
index 427dc74..e9313b3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java
@@ -22,7 +22,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.runtime.operators.AllGroupReduceDriver;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java
index b124e51..0a0f44e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java
@@ -23,7 +23,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.runtime.operators.AllReduceDriver;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java
index a29d4e0..4dc64b1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java
index ae9c294..74b96d8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java
@@ -24,7 +24,7 @@ import java.util.List;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java
index 28217b4..52ac4b0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
index 666ecd5..8d33a98 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordComparator;
 import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactory;
 import org.apache.flink.configuration.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
index 1d7cfa6..f58e360 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
@@ -23,9 +23,9 @@ import static org.junit.Assert.fail;
 
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichJoinFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.runtime.operators.DriverStrategy;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
index 1d0b765..21798bb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
index e4faa77..06a8eaf 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
@@ -21,9 +21,9 @@ package org.apache.flink.test.iterative;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.flink.api.java.functions.RichCoGroupFunction;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
index a0d26db..2d53f7b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
@@ -21,9 +21,9 @@ package org.apache.flink.test.iterative;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.functions.RichJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.JavaProgramTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
index 12eb87b..98fa54a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
@@ -28,8 +28,8 @@ import org.junit.Assert;
 
 import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
 import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/568dff12/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
index a9f01fb..1349040 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
@@ -23,8 +23,8 @@ import java.util.List;
 
 import org.apache.flink.api.common.aggregators.LongSumAggregator;
 import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;


[15/60] git commit: Refactor Tuple wrapping/unwrapping in Join and CoGroup

Posted by al...@apache.org.
Refactor Tuple wrapping/unwrapping in Join and CoGroup

Previously both sides were always wrapped/unwrapped, even if one side
was a tuple type. This was not compatible with the Scala tuples and is
also not necessary anymore because of the GenericPairComparator.


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

Branch: refs/heads/master
Commit: a32890aeefe409244cf5575b906ddcbaa0bcf086
Parents: 16dad15
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 2 18:41:53 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../api/java/operators/CoGroupOperator.java     | 160 +++++++------
 .../flink/api/java/operators/JoinOperator.java  | 224 +++++++++++--------
 .../apache/flink/api/java/operators/Keys.java   |   6 +-
 .../PlanBothUnwrappingCoGroupOperator.java      |  82 +++++++
 .../PlanBothUnwrappingJoinOperator.java         |  66 ++++++
 .../PlanLeftUnwrappingCoGroupOperator.java      |  78 +++++++
 .../PlanLeftUnwrappingJoinOperator.java         |  64 ++++++
 .../PlanRightUnwrappingCoGroupOperator.java     |  78 +++++++
 .../PlanRightUnwrappingJoinOperator.java        |  66 ++++++
 .../PlanUnwrappingCoGroupOperator.java          |  86 -------
 .../translation/PlanUnwrappingJoinOperator.java |  77 -------
 .../translation/TupleKeyExtractingMapper.java   |  57 -----
 12 files changed, 654 insertions(+), 390 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
index dd0eba1..81f29e2 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
@@ -35,8 +35,9 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
 import org.apache.flink.api.java.operators.Keys.FieldPositionKeys;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
-import org.apache.flink.api.java.operators.translation.PlanUnwrappingCoGroupOperator;
-import org.apache.flink.api.java.operators.translation.TupleKeyExtractingMapper;
+import org.apache.flink.api.java.operators.translation.PlanBothUnwrappingCoGroupOperator;
+import org.apache.flink.api.java.operators.translation.PlanLeftUnwrappingCoGroupOperator;
+import org.apache.flink.api.java.operators.translation.PlanRightUnwrappingCoGroupOperator;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
@@ -60,7 +61,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 	private final Keys<I2> keys2;
 
 
-	protected CoGroupOperator(DataSet<I1> input1, DataSet<I2> input2,
+	public CoGroupOperator(DataSet<I1> input1, DataSet<I2> input2,
 							Keys<I1> keys1, Keys<I2> keys2,
 							CoGroupFunction<I1, I2, OUT> function,
 							TypeInformation<OUT> returnType)
@@ -92,16 +93,22 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		
 		String name = getName() != null ? getName() : function.getClass().getName();
 
+		if (!keys1.areCompatibale(keys2)) {
+			throw new InvalidProgramException("The types of the key fields do not match. Left:" +
+					" " + keys1 + " Right: " + keys2);
+		}
+
 		if (keys1 instanceof Keys.SelectorFunctionKeys
-				&& keys2 instanceof Keys.SelectorFunctionKeys
-				&& keys1.areCompatibale(keys2)) {
+				&& keys2 instanceof Keys.SelectorFunctionKeys) {
 
 			@SuppressWarnings("unchecked")
-			Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) keys1;
+			Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 =
+					(Keys.SelectorFunctionKeys<I1, ?>) keys1;
 			@SuppressWarnings("unchecked")
-			Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) keys2;
+			Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 =
+					(Keys.SelectorFunctionKeys<I2, ?>) keys2;
 
-			PlanUnwrappingCoGroupOperator<I1, I2, OUT, ?> po =
+			PlanBothUnwrappingCoGroupOperator<I1, I2, OUT, ?> po =
 					translateSelectorFunctionCoGroup(selectorKeys1, selectorKeys2, function,
 					getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
 
@@ -111,6 +118,38 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			return po;
 
 		}
+		else if (keys2 instanceof Keys.SelectorFunctionKeys) {
+
+			int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions();
+
+			@SuppressWarnings("unchecked")
+			Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) keys2;
+
+			PlanRightUnwrappingCoGroupOperator<I1, I2, OUT, ?> po =
+					translateSelectorFunctionCoGroupRight(logicalKeyPositions1, selectorKeys2, function,
+							getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
+
+			// set dop
+			po.setDegreeOfParallelism(this.getParallelism());
+
+			return po;
+		}
+		else if (keys1 instanceof Keys.SelectorFunctionKeys) {
+
+			@SuppressWarnings("unchecked")
+			Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) keys1;
+
+			int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions();
+
+			PlanLeftUnwrappingCoGroupOperator<I1, I2, OUT, ?> po =
+					translateSelectorFunctionCoGroupLeft(selectorKeys1, logicalKeyPositions2, function,
+							getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
+
+			// set dop
+			po.setDegreeOfParallelism(this.getParallelism());
+
+			return po;
+		}
 		else if ((keys1 instanceof Keys.FieldPositionKeys
 				&& keys2 instanceof Keys.FieldPositionKeys) ||
 				((keys1 instanceof Keys.ExpressionKeys
@@ -139,51 +178,13 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			return po;
 
 		}
-		else if (keys1 instanceof Keys.FieldPositionKeys
-				&& keys2 instanceof Keys.SelectorFunctionKeys
-				&& keys1.areCompatibale(keys2)
-			) {
-
-			int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions();
-
-			@SuppressWarnings("unchecked")
-			Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) keys2;
-
-			PlanUnwrappingCoGroupOperator<I1, I2, OUT, ?> po =
-					translateSelectorFunctionCoGroupRight(logicalKeyPositions1, selectorKeys2, function,
-					getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
-
-			// set dop
-			po.setDegreeOfParallelism(this.getParallelism());
-
-			return po;
-		}
-		else if (keys1 instanceof Keys.SelectorFunctionKeys
-				&& keys2 instanceof Keys.FieldPositionKeys
-				&& keys1.areCompatibale(keys2)
-			) {
-
-			@SuppressWarnings("unchecked")
-			Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) keys1;
-
-			int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions();
-
-			PlanUnwrappingCoGroupOperator<I1, I2, OUT, ?> po =
-					translateSelectorFunctionCoGroupLeft(selectorKeys1, logicalKeyPositions2, function,
-					getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
-
-			// set dop
-			po.setDegreeOfParallelism(this.getParallelism());
-
-			return po;
-		}
 		else {
 			throw new UnsupportedOperationException("Unrecognized or incompatible key types.");
 		}
 	}
 
 
-	private static <I1, I2, K, OUT> PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> translateSelectorFunctionCoGroup(
+	private static <I1, I2, K, OUT> PlanBothUnwrappingCoGroupOperator<I1, I2, OUT, K> translateSelectorFunctionCoGroup(
 			Keys.SelectorFunctionKeys<I1, ?> rawKeys1, Keys.SelectorFunctionKeys<I2, ?> rawKeys2,
 			CoGroupFunction<I1, I2, OUT> function,
 			TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
@@ -204,7 +205,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
 		final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
 				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
-		final PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
+		final PlanBothUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanBothUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
 
 		cogroup.setFirstInput(keyMapper1);
 		cogroup.setSecondInput(keyMapper2);
@@ -218,7 +219,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		return cogroup;
 	}
 
-	private static <I1, I2, K, OUT> PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> translateSelectorFunctionCoGroupRight(
+	private static <I1, I2, K, OUT> PlanRightUnwrappingCoGroupOperator<I1, I2, OUT, K> translateSelectorFunctionCoGroupRight(
 			int[] logicalKeyPositions1, Keys.SelectorFunctionKeys<I2, ?> rawKeys2,
 			CoGroupFunction<I1, I2, OUT> function,
 			TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
@@ -231,32 +232,39 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		@SuppressWarnings("unchecked")
 		final Keys.SelectorFunctionKeys<I2, K> keys2 = (Keys.SelectorFunctionKeys<I2, K>) rawKeys2;
 
-		final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 = new TupleTypeInfo<Tuple2<K, I1>>(keys2.getKeyType(), inputType1); // assume same key, checked by Key.areCompatibale() before
-		final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 = new TupleTypeInfo<Tuple2<K, I2>>(keys2.getKeyType(), inputType2);
+		final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 =
+				new TupleTypeInfo<Tuple2<K, I2>>(keys2.getKeyType(), inputType2);
 
-		final TupleKeyExtractingMapper<I1, K> extractor1 = new TupleKeyExtractingMapper<I1, K>(logicalKeyPositions1[0]);
-		final KeyExtractingMapper<I2, K> extractor2 = new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
+		final KeyExtractingMapper<I2, K> extractor2 =
+				new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
 
-		final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
-				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
 		final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
-				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
+				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(
+						extractor2,
+						new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2),
+						"Key Extractor 2");
 		
-		final PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
-
-		cogroup.setFirstInput(keyMapper1);
+		final PlanRightUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup =
+				new PlanRightUnwrappingCoGroupOperator<I1, I2, OUT, K>(
+						function,
+						logicalKeyPositions1,
+						keys2,
+						name,
+						outputType,
+						inputType1,
+						typeInfoWithKey2);
+
+		cogroup.setFirstInput(input1);
 		cogroup.setSecondInput(keyMapper2);
 
-		keyMapper1.setInput(input1);
 		keyMapper2.setInput(input2);
 		// set dop
-		keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
 		keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
 
 		return cogroup;
 	}
 
-	private static <I1, I2, K, OUT> PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> translateSelectorFunctionCoGroupLeft(
+	private static <I1, I2, K, OUT> PlanLeftUnwrappingCoGroupOperator<I1, I2, OUT, K> translateSelectorFunctionCoGroupLeft(
 			Keys.SelectorFunctionKeys<I1, ?> rawKeys1, int[] logicalKeyPositions2,
 			CoGroupFunction<I1, I2, OUT> function,
 			TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
@@ -269,27 +277,33 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		@SuppressWarnings("unchecked")
 		final Keys.SelectorFunctionKeys<I1, K> keys1 = (Keys.SelectorFunctionKeys<I1, K>) rawKeys1;
 
-		final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 = new TupleTypeInfo<Tuple2<K, I1>>(keys1.getKeyType(), inputType1); // assume same key, checked by Key.areCompatibale() before
-		final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 = new TupleTypeInfo<Tuple2<K, I2>>(keys1.getKeyType(), inputType2);
+		final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 =
+				new TupleTypeInfo<Tuple2<K, I1>>(keys1.getKeyType(), inputType1);
 
 		final KeyExtractingMapper<I1, K> extractor1 = new KeyExtractingMapper<I1, K>(keys1.getKeyExtractor());
-		final TupleKeyExtractingMapper<I2, K> extractor2 = new TupleKeyExtractingMapper<I2, K>(logicalKeyPositions2[0]);
 
 		final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
-				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
-		final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
-				new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
-		
-		final PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup = new PlanUnwrappingCoGroupOperator<I1, I2, OUT, K>(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
+				new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(
+						extractor1,
+						new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1),
+						"Key Extractor 1");
+
+		final PlanLeftUnwrappingCoGroupOperator<I1, I2, OUT, K> cogroup =
+				new PlanLeftUnwrappingCoGroupOperator<I1, I2, OUT, K>(
+						function,
+						keys1,
+						logicalKeyPositions2,
+						name,
+						outputType,
+						typeInfoWithKey1,
+						inputType2);
 
 		cogroup.setFirstInput(keyMapper1);
-		cogroup.setSecondInput(keyMapper2);
+		cogroup.setSecondInput(input2);
 
 		keyMapper1.setInput(input1);
-		keyMapper2.setInput(input2);
 		// set dop
 		keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
-		keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
 
 		return cogroup;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index 6d74474..2875251 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -40,8 +40,9 @@ import org.apache.flink.api.java.functions.SemanticPropUtil;
 import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
 import org.apache.flink.api.java.operators.Keys.FieldPositionKeys;
 import org.apache.flink.api.java.operators.translation.KeyExtractingMapper;
-import org.apache.flink.api.java.operators.translation.PlanUnwrappingJoinOperator;
-import org.apache.flink.api.java.operators.translation.TupleKeyExtractingMapper;
+import org.apache.flink.api.java.operators.translation.PlanBothUnwrappingJoinOperator;
+import org.apache.flink.api.java.operators.translation.PlanLeftUnwrappingJoinOperator;
+import org.apache.flink.api.java.operators.translation.PlanRightUnwrappingJoinOperator;
 import org.apache.flink.api.java.operators.translation.WrappingFunction;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -108,8 +109,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		REPARTITION_SORT_MERGE,
 	};
 	
-	private final Keys<I1> keys1;
-	private final Keys<I2> keys2;
+	protected final Keys<I1> keys1;
+	protected final Keys<I2> keys2;
 	
 	private JoinHint joinHint;
 	
@@ -227,20 +228,30 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 //		}
 		
 		@Override
-		protected org.apache.flink.api.common.operators.base.JoinOperatorBase<?, ?, OUT, ?> translateToDataFlow(Operator<I1> input1, Operator<I2> input2) {
+		protected JoinOperatorBase<?, ?, OUT, ?> translateToDataFlow(
+				Operator<I1> input1,
+				Operator<I2> input2) {
 			
 			String name = getName() != null ? getName() : function.getClass().getName();
-			
-			if (super.keys1 instanceof Keys.SelectorFunctionKeys 
-					&& super.keys2 instanceof Keys.SelectorFunctionKeys
-					&& super.keys1.areCompatibale(super.keys2)) {
-				
+
+			if (!keys1.areCompatibale(keys2)) {
+				throw new InvalidProgramException("The types of the key fields do not match. Left:" +
+						" " + keys1 + " Right: " + keys2);
+			}
+
+			if (keys1 instanceof Keys.SelectorFunctionKeys
+					&& keys2 instanceof Keys.SelectorFunctionKeys) {
+				// Both join sides have a key selector function, so we need to do the
+				// tuple wrapping/unwrapping on both sides.
+
 				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) super.keys1;
+				Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 =
+						(Keys.SelectorFunctionKeys<I1, ?>) keys1;
 				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) super.keys2;
+				Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 =
+						(Keys.SelectorFunctionKeys<I2, ?>) keys2;
 				
-				PlanUnwrappingJoinOperator<I1, I2, OUT, ?> po = 
+				PlanBothUnwrappingJoinOperator<I1, I2, OUT, ?> po =
 						translateSelectorFunctionJoin(selectorKeys1, selectorKeys2, function, 
 						getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
 				
@@ -250,15 +261,51 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 				return po;
 				
 			}
+			else if (keys2 instanceof Keys.SelectorFunctionKeys) {
+				// The right side of the join needs the tuple wrapping/unwrapping
+
+				int[] logicalKeyPositions1 = keys1.computeLogicalKeyPositions();
+
+				@SuppressWarnings("unchecked")
+				Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 =
+						(Keys.SelectorFunctionKeys<I2, ?>) keys2;
+
+				PlanRightUnwrappingJoinOperator<I1, I2, OUT, ?> po =
+						translateSelectorFunctionJoinRight(logicalKeyPositions1, selectorKeys2,
+								function, getInput1Type(), getInput2Type(), getResultType(), name,
+								input1, input2);
+
+				// set dop
+				po.setDegreeOfParallelism(this.getParallelism());
+
+				return po;
+			}
+			else if (keys1 instanceof Keys.SelectorFunctionKeys) {
+				// The left side of the join needs the tuple wrapping/unwrapping
+
+
+				@SuppressWarnings("unchecked")
+				Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 =
+						(Keys.SelectorFunctionKeys<I1, ?>) keys1;
+
+				int[] logicalKeyPositions2 = keys2.computeLogicalKeyPositions();
+
+				PlanLeftUnwrappingJoinOperator<I1, I2, OUT, ?> po =
+						translateSelectorFunctionJoinLeft(selectorKeys1, logicalKeyPositions2, function,
+								getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
+
+				// set dop
+				po.setDegreeOfParallelism(this.getParallelism());
+
+				return po;
+			}
 			else if ((super.keys1 instanceof Keys.FieldPositionKeys
 						&& super.keys2 instanceof Keys.FieldPositionKeys) ||
 					((super.keys1 instanceof Keys.ExpressionKeys
 							&& super.keys2 instanceof Keys.ExpressionKeys)))
 			{
-				if (!super.keys1.areCompatibale(super.keys2)) {
-					throw new InvalidProgramException("The types of the key fields do not match.");
-				}
-				
+				// Neither side needs the tuple wrapping/unwrapping
+
 				int[] logicalKeyPositions1 = super.keys1.computeLogicalKeyPositions();
 				int[] logicalKeyPositions2 = super.keys2.computeLogicalKeyPositions();
 				
@@ -276,51 +323,13 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 				
 				return po;
 			}
-			else if (super.keys1 instanceof Keys.FieldPositionKeys 
-					&& super.keys2 instanceof Keys.SelectorFunctionKeys
-					&& super.keys1.areCompatibale(super.keys2)
-				) {
-			
-				int[] logicalKeyPositions1 = super.keys1.computeLogicalKeyPositions();
-				
-				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I2, ?> selectorKeys2 = (Keys.SelectorFunctionKeys<I2, ?>) super.keys2;
-				
-				PlanUnwrappingJoinOperator<I1, I2, OUT, ?> po = 
-						translateSelectorFunctionJoinRight(logicalKeyPositions1, selectorKeys2, function, 
-						getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
-				
-				// set dop
-				po.setDegreeOfParallelism(this.getParallelism());
-				
-				return po;
-			}
-			else if (super.keys1 instanceof Keys.SelectorFunctionKeys
-					&& super.keys2 instanceof Keys.FieldPositionKeys 
-					&& super.keys1.areCompatibale(super.keys2)
-				) {
-				
-				@SuppressWarnings("unchecked")
-				Keys.SelectorFunctionKeys<I1, ?> selectorKeys1 = (Keys.SelectorFunctionKeys<I1, ?>) super.keys1;
-				
-				int[] logicalKeyPositions2 = super.keys2.computeLogicalKeyPositions();
-				
-				PlanUnwrappingJoinOperator<I1, I2, OUT, ?> po =
-						translateSelectorFunctionJoinLeft(selectorKeys1, logicalKeyPositions2, function, 
-						getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
-				
-				// set dop
-				po.setDegreeOfParallelism(this.getParallelism());
-				
-				return po;
-			}
 			else {
 				throw new UnsupportedOperationException("Unrecognized or incompatible key types.");
 			}
 			
 		}
 		
-		private static <I1, I2, K, OUT> PlanUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoin(
+		private static <I1, I2, K, OUT> PlanBothUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoin(
 				Keys.SelectorFunctionKeys<I1, ?> rawKeys1, Keys.SelectorFunctionKeys<I2, ?> rawKeys2, 
 				FlatJoinFunction<I1, I2, OUT> function,
 				TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
@@ -341,7 +350,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 					new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
 			final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
 					new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
-			final PlanUnwrappingJoinOperator<I1, I2, OUT, K> join = new PlanUnwrappingJoinOperator<I1, I2, OUT, K>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
+			final PlanBothUnwrappingJoinOperator<I1, I2, OUT, K> join = new PlanBothUnwrappingJoinOperator<I1, I2, OUT, K>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
 			
 			join.setFirstInput(keyMapper1);
 			join.setSecondInput(keyMapper2);
@@ -355,50 +364,68 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			return join;
 		}
 		
-		private static <I1, I2, K, OUT> PlanUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoinRight(
-				int[] logicalKeyPositions1, Keys.SelectorFunctionKeys<I2, ?> rawKeys2, 
+		private static <I1, I2, K, OUT> PlanRightUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoinRight(
+				int[] logicalKeyPositions1,
+				Keys.SelectorFunctionKeys<I2, ?> rawKeys2,
 				FlatJoinFunction<I1, I2, OUT> function,
-				TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
-				Operator<I1> input1, Operator<I2> input2)
-		{
+				TypeInformation<I1> inputType1,
+				TypeInformation<I2> inputType2,
+				TypeInformation<OUT> outputType,
+				String name,
+				Operator<I1> input1,
+				Operator<I2> input2) {
+
 			if(!inputType1.isTupleType()) {
 				throw new InvalidParameterException("Should not happen.");
 			}
 			
 			@SuppressWarnings("unchecked")
-			final Keys.SelectorFunctionKeys<I2, K> keys2 = (Keys.SelectorFunctionKeys<I2, K>) rawKeys2;
+			final Keys.SelectorFunctionKeys<I2, K> keys2 =
+					(Keys.SelectorFunctionKeys<I2, K>) rawKeys2;
 			
-			final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 = new TupleTypeInfo<Tuple2<K, I1>>(keys2.getKeyType(), inputType1); // assume same key, checked by Key.areCompatibale() before
-			final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 = new TupleTypeInfo<Tuple2<K, I2>>(keys2.getKeyType(), inputType2);
+			final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 =
+					new TupleTypeInfo<Tuple2<K, I2>>(keys2.getKeyType(), inputType2);
 			
-			final TupleKeyExtractingMapper<I1, K> extractor1 = new TupleKeyExtractingMapper<I1, K>(logicalKeyPositions1[0]);
-			final KeyExtractingMapper<I2, K> extractor2 = new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
+			final KeyExtractingMapper<I2, K> extractor2 =
+					new KeyExtractingMapper<I2, K>(keys2.getKeyExtractor());
 
-			final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
-					new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
 			final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
-					new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
-			
-			final PlanUnwrappingJoinOperator<I1, I2, OUT, K> join = new PlanUnwrappingJoinOperator<I1, I2, OUT, K>(function, logicalKeyPositions1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
-			
-			join.setFirstInput(keyMapper1);
+					new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(
+							extractor2,
+							new UnaryOperatorInformation<I2,Tuple2<K, I2>>(inputType2, typeInfoWithKey2),
+							"Key Extractor 2");
+			
+			final PlanRightUnwrappingJoinOperator<I1, I2, OUT, K> join =
+					new PlanRightUnwrappingJoinOperator<I1, I2, OUT, K>(
+							function,
+							logicalKeyPositions1,
+							keys2,
+							name,
+							outputType,
+							inputType1,
+							typeInfoWithKey2);
+			
+			join.setFirstInput(input1);
 			join.setSecondInput(keyMapper2);
 			
-			keyMapper1.setInput(input1);
 			keyMapper2.setInput(input2);
 			// set dop
-			keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
 			keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
 			
 			return join;
 		}
 		
-		private static <I1, I2, K, OUT> PlanUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoinLeft(
-				Keys.SelectorFunctionKeys<I1, ?> rawKeys1, int[] logicalKeyPositions2,
+		private static <I1, I2, K, OUT> PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K> translateSelectorFunctionJoinLeft(
+				Keys.SelectorFunctionKeys<I1, ?> rawKeys1,
+				int[] logicalKeyPositions2,
 				FlatJoinFunction<I1, I2, OUT> function,
-				TypeInformation<I1> inputType1, TypeInformation<I2> inputType2, TypeInformation<OUT> outputType, String name,
-				Operator<I1> input1, Operator<I2> input2)
-		{
+				TypeInformation<I1> inputType1,
+				TypeInformation<I2> inputType2,
+				TypeInformation<OUT> outputType,
+				String name,
+				Operator<I1> input1,
+				Operator<I2> input2) {
+
 			if(!inputType2.isTupleType()) {
 				throw new InvalidParameterException("Should not happen.");
 			}
@@ -406,28 +433,35 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			@SuppressWarnings("unchecked")
 			final Keys.SelectorFunctionKeys<I1, K> keys1 = (Keys.SelectorFunctionKeys<I1, K>) rawKeys1;
 			
-			final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 = new TupleTypeInfo<Tuple2<K, I1>>(keys1.getKeyType(), inputType1); // assume same key, checked by Key.areCompatibale() before
-			final TypeInformation<Tuple2<K, I2>> typeInfoWithKey2 = new TupleTypeInfo<Tuple2<K, I2>>(keys1.getKeyType(), inputType2);
-			
-			final KeyExtractingMapper<I1, K> extractor1 = new KeyExtractingMapper<I1, K>(keys1.getKeyExtractor());
-			final TupleKeyExtractingMapper<I2, K> extractor2 = new TupleKeyExtractingMapper<I2, K>(logicalKeyPositions2[0]);
+			final TypeInformation<Tuple2<K, I1>> typeInfoWithKey1 =
+					new TupleTypeInfo<Tuple2<K, I1>>(keys1.getKeyType(), inputType1);
+
+			final KeyExtractingMapper<I1, K> extractor1 =
+					new KeyExtractingMapper<I1, K>(keys1.getKeyExtractor());
 
 			final MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>> keyMapper1 =
-					new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(extractor1, new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1), "Key Extractor 1");
-			final MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>> keyMapper2 =
-					new MapOperatorBase<I2, Tuple2<K, I2>, MapFunction<I2, Tuple2<K, I2>>>(extractor2, new UnaryOperatorInformation<I2, Tuple2<K, I2>>(inputType2, typeInfoWithKey2), "Key Extractor 2");
-			
-			final PlanUnwrappingJoinOperator<I1, I2, OUT, K> join = new PlanUnwrappingJoinOperator<I1, I2, OUT, K>(function, keys1, logicalKeyPositions2, name, outputType, typeInfoWithKey1, typeInfoWithKey2);
+					new MapOperatorBase<I1, Tuple2<K, I1>, MapFunction<I1, Tuple2<K, I1>>>(
+							extractor1,
+							new UnaryOperatorInformation<I1, Tuple2<K, I1>>(inputType1, typeInfoWithKey1),
+							"Key Extractor 1");
+
+			final PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K> join =
+					new PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K>(
+							function,
+							keys1,
+							logicalKeyPositions2,
+							name,
+							outputType,
+							typeInfoWithKey1,
+							inputType2);
 			
 			join.setFirstInput(keyMapper1);
-			join.setSecondInput(keyMapper2);
+			join.setSecondInput(input2);
 			
 			keyMapper1.setInput(input1);
-			keyMapper2.setInput(input2);
 			// set dop
 			keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
-			keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
-			
+
 			return join;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
index 630f674..125ad85 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
@@ -115,7 +115,9 @@ public abstract class Keys<T> {
 
 		@Override
 		public String toString() {
-			return Arrays.toString(fieldPositions);
+			String fieldsString = Arrays.toString(fieldPositions);
+			String typesString = Arrays.toString(types);
+			return "Tuple position key (Fields: " + fieldsString + " Types: " + typesString + ")";
 		}
 	}
 
@@ -182,7 +184,7 @@ public abstract class Keys<T> {
 
 		@Override
 		public String toString() {
-			return keyExtractor + " (" + keyType + ")";
+			return "Key function (Type: " + keyType + ")";
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java
new file mode 100644
index 0000000..3a83245
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+
+public class PlanBothUnwrappingCoGroupOperator<I1, I2, OUT, K>
+		extends CoGroupOperatorBase<Tuple2<K, I1>, Tuple2<K, I2>, OUT, CoGroupFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>>
+{
+
+	public PlanBothUnwrappingCoGroupOperator(
+			CoGroupFunction<I1, I2, OUT> udf,
+			Keys.SelectorFunctionKeys<I1, K> key1,
+			Keys.SelectorFunctionKeys<I2, K> key2,
+			String name,
+			TypeInformation<OUT> type,
+			TypeInformation<Tuple2<K, I1>> typeInfoWithKey1,
+			TypeInformation<Tuple2<K, I2>> typeInfoWithKey2) {
+
+		super(
+				new TupleBothUnwrappingCoGrouper<I1, I2, OUT, K>(udf),
+				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(
+						typeInfoWithKey1,
+						typeInfoWithKey2,
+						type),
+				key1.computeLogicalKeyPositions(),
+				key2.computeLogicalKeyPositions(),
+				name);
+	}
+
+	public static final class TupleBothUnwrappingCoGrouper<I1, I2, OUT, K>
+			extends WrappingFunction<CoGroupFunction<I1, I2, OUT>>
+			implements CoGroupFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>
+	{
+		private static final long serialVersionUID = 1L;
+		
+		private final TupleUnwrappingIterator<I1, K> iter1;
+		private final TupleUnwrappingIterator<I2, K> iter2;
+		
+		private TupleBothUnwrappingCoGrouper(CoGroupFunction<I1, I2, OUT> wrapped) {
+			super(wrapped);
+			
+			this.iter1 = new TupleUnwrappingIterator<I1, K>();
+			this.iter2 = new TupleUnwrappingIterator<I2, K>();
+		}
+
+
+		@Override
+		public void coGroup(
+				Iterable<Tuple2<K, I1>> records1,
+				Iterable<Tuple2<K, I2>> records2,
+				Collector<OUT> out) throws Exception {
+
+			iter1.set(records1.iterator());
+			iter2.set(records2.iterator());
+			this.wrappedFunction.coGroup(iter1, iter2, out);
+		}
+		
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java
new file mode 100644
index 0000000..3f89493
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingJoinOperator.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+
+public class PlanBothUnwrappingJoinOperator<I1, I2, OUT, K>
+	extends JoinOperatorBase<Tuple2<K, I1>, Tuple2<K, I2>, OUT, FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>>
+{
+
+	public PlanBothUnwrappingJoinOperator(
+			FlatJoinFunction<I1, I2, OUT> udf,
+			Keys.SelectorFunctionKeys<I1, K> key1,
+			Keys.SelectorFunctionKeys<I2, K> key2, String name,
+			TypeInformation<OUT> resultType,
+			TypeInformation<Tuple2<K, I1>> typeInfoWithKey1,
+			TypeInformation<Tuple2<K, I2>> typeInfoWithKey2) {
+
+		super(
+				new TupleUnwrappingJoiner<I1, I2, OUT, K>(udf),
+				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(
+						typeInfoWithKey1,
+						typeInfoWithKey2,
+						resultType),
+				key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name);
+	}
+
+	public static final class TupleUnwrappingJoiner<I1, I2, OUT, K>
+		extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
+		implements FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT> {
+
+		private static final long serialVersionUID = 1L;
+		
+		private TupleUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
+			super(wrapped);
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public void join (Tuple2<K, I1> value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
+			wrappedFunction.join ((I1)(value1.getField(1)), (I2)(value2.getField(1)), collector);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java
new file mode 100644
index 0000000..303c50d
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+
+public class PlanLeftUnwrappingCoGroupOperator<I1, I2, OUT, K>
+		extends CoGroupOperatorBase<Tuple2<K, I1>, I2, OUT, CoGroupFunction<Tuple2<K, I1>, I2, OUT>>
+{
+
+	public PlanLeftUnwrappingCoGroupOperator(
+			CoGroupFunction<I1, I2, OUT> udf,
+			Keys.SelectorFunctionKeys<I1, K> key1,
+			int[] key2,
+			String name,
+			TypeInformation<OUT> resultType,
+			TypeInformation<Tuple2<K, I1>> typeInfoWithKey1,
+			TypeInformation<I2> typeInfo2) {
+
+		super(
+				new TupleLeftUnwrappingCoGrouper<I1, I2, OUT, K>(udf),
+				new BinaryOperatorInformation<Tuple2<K, I1>, I2, OUT>(
+						typeInfoWithKey1,
+						typeInfo2,
+						resultType),
+				key1.computeLogicalKeyPositions(),
+				key2,
+				name);
+	}
+
+	public static final class TupleLeftUnwrappingCoGrouper<I1, I2, OUT, K>
+			extends WrappingFunction<CoGroupFunction<I1, I2, OUT>>
+			implements CoGroupFunction<Tuple2<K, I1>, I2, OUT> {
+
+		private static final long serialVersionUID = 1L;
+		
+		private final TupleUnwrappingIterator<I1, K> iter1;
+
+		private TupleLeftUnwrappingCoGrouper(CoGroupFunction<I1, I2, OUT> wrapped) {
+			super(wrapped);
+			
+			this.iter1 = new TupleUnwrappingIterator<I1, K>();
+		}
+
+
+		@Override
+		public void coGroup(
+				Iterable<Tuple2<K, I1>> records1,
+				Iterable<I2> records2,
+				Collector<OUT> out) throws Exception {
+
+			iter1.set(records1.iterator());
+			this.wrappedFunction.coGroup(iter1, records2, out);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java
new file mode 100644
index 0000000..6e92d44
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingJoinOperator.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+
+public class PlanLeftUnwrappingJoinOperator<I1, I2, OUT, K>
+		extends JoinOperatorBase<Tuple2<K, I1>, I2, OUT, FlatJoinFunction<Tuple2<K, I1>, I2, OUT>> {
+
+	public PlanLeftUnwrappingJoinOperator(
+			FlatJoinFunction<I1, I2, OUT> udf,
+			Keys.SelectorFunctionKeys<I1, K> key1,
+			int[] key2, String name,
+			TypeInformation<OUT> resultType,
+			TypeInformation<Tuple2<K, I1>> typeInfoWithKey1,
+			TypeInformation<I2> typeInfo2) {
+		super(
+				new TupleLeftUnwrappingJoiner<I1, I2, OUT, K>(udf),
+				new BinaryOperatorInformation<Tuple2<K, I1>, I2, OUT>(
+						typeInfoWithKey1,
+						typeInfo2,
+						resultType),
+				key1.computeLogicalKeyPositions(), key2, name);
+	}
+
+	public static final class TupleLeftUnwrappingJoiner<I1, I2, OUT, K>
+			extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
+			implements FlatJoinFunction<Tuple2<K, I1>, I2, OUT> {
+
+		private static final long serialVersionUID = 1L;
+
+		private TupleLeftUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
+			super(wrapped);
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public void join (Tuple2<K, I1> value1, I2 value2, Collector<OUT> collector) throws Exception {
+			wrappedFunction.join ((I1)(value1.getField(1)), value2, collector);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java
new file mode 100644
index 0000000..894a4a2
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+
+public class PlanRightUnwrappingCoGroupOperator<I1, I2, OUT, K>
+		extends CoGroupOperatorBase<I1, Tuple2<K, I2>, OUT, CoGroupFunction<I1, Tuple2<K, I2>, OUT>>
+{
+
+	public PlanRightUnwrappingCoGroupOperator(
+			CoGroupFunction<I1, I2, OUT> udf,
+			int[] key1,
+			Keys.SelectorFunctionKeys<I2, K> key2,
+			String name,
+			TypeInformation<OUT> resultType,
+			TypeInformation<I1> typeInfo1,
+			TypeInformation<Tuple2<K, I2>> typeInfoWithKey2) {
+
+		super(
+				new TupleRightUnwrappingCoGrouper<I1, I2, OUT, K>(udf),
+				new BinaryOperatorInformation<I1, Tuple2<K, I2>, OUT>(
+						typeInfo1,
+						typeInfoWithKey2,
+						resultType),
+				key1,
+				key2.computeLogicalKeyPositions(),
+				name);
+	}
+
+	public static final class TupleRightUnwrappingCoGrouper<I1, I2, OUT, K>
+			extends WrappingFunction<CoGroupFunction<I1, I2, OUT>>
+			implements CoGroupFunction<I1, Tuple2<K, I2>, OUT> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final TupleUnwrappingIterator<I2, K> iter2;
+
+		private TupleRightUnwrappingCoGrouper(CoGroupFunction<I1, I2, OUT> wrapped) {
+			super(wrapped);
+
+			this.iter2 = new TupleUnwrappingIterator<I2, K>();
+		}
+
+
+		@Override
+		public void coGroup(
+				Iterable<I1> records1,
+				Iterable<Tuple2<K, I2>> records2,
+				Collector<OUT> out) throws Exception {
+
+			iter2.set(records2.iterator());
+			this.wrappedFunction.coGroup(records1, iter2, out);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java
new file mode 100644
index 0000000..c2973b7
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingJoinOperator.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.operators.BinaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.util.Collector;
+
+public class PlanRightUnwrappingJoinOperator<I1, I2, OUT, K>
+		extends JoinOperatorBase<I1, Tuple2<K, I2>, OUT, FlatJoinFunction<I1, Tuple2<K, I2>, OUT>> {
+
+	public PlanRightUnwrappingJoinOperator(
+			FlatJoinFunction<I1, I2, OUT> udf,
+			int[] key1,
+			Keys.SelectorFunctionKeys<I2, K> key2,
+			String name,
+			TypeInformation<OUT> type,
+			TypeInformation<I1> typeInfo1,
+			TypeInformation<Tuple2<K, I2>> typeInfoWithKey2) {
+
+		super(
+				new TupleRightUnwrappingJoiner<I1, I2, OUT, K>(udf),
+				new BinaryOperatorInformation<I1, Tuple2<K, I2>, OUT>(
+						typeInfo1,
+						typeInfoWithKey2,
+						type),
+				key1, key2.computeLogicalKeyPositions(), name);
+	}
+
+	public static final class TupleRightUnwrappingJoiner<I1, I2, OUT, K>
+			extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
+			implements FlatJoinFunction<I1, Tuple2<K, I2>, OUT> {
+
+		private static final long serialVersionUID = 1L;
+
+		private TupleRightUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
+			super(wrapped);
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public void join (I1 value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
+			wrappedFunction.join (value1, (I2)(value2.getField(1)), collector);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java
deleted file mode 100644
index 36bae6b..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingCoGroupOperator.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.operators.translation;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.operators.BinaryOperatorInformation;
-import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.types.TypeInformation;
-import org.apache.flink.util.Collector;
-
-public class PlanUnwrappingCoGroupOperator<I1, I2, OUT, K> 
-	extends CoGroupOperatorBase<Tuple2<K, I1>, Tuple2<K, I2>, OUT, CoGroupFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>>
-{
-
-	public PlanUnwrappingCoGroupOperator(CoGroupFunction<I1, I2, OUT> udf,
-			Keys.SelectorFunctionKeys<I1, K> key1, Keys.SelectorFunctionKeys<I2, K> key2, String name,
-			TypeInformation<OUT> type, TypeInformation<Tuple2<K, I1>> typeInfoWithKey1, TypeInformation<Tuple2<K, I2>> typeInfoWithKey2)
-	{
-		super(new TupleUnwrappingCoGrouper<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(typeInfoWithKey1, typeInfoWithKey2, type),
-				key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name);
-	}
-	
-	public PlanUnwrappingCoGroupOperator(CoGroupFunction<I1, I2, OUT> udf,
-			int[] key1, Keys.SelectorFunctionKeys<I2, K> key2, String name,
-			TypeInformation<OUT> type, TypeInformation<Tuple2<K, I1>> typeInfoWithKey1, TypeInformation<Tuple2<K, I2>> typeInfoWithKey2)
-	{
-		super(new TupleUnwrappingCoGrouper<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(typeInfoWithKey1,typeInfoWithKey2, type),
-				new int[]{0}, key2.computeLogicalKeyPositions(), name);
-	}
-	
-	public PlanUnwrappingCoGroupOperator(CoGroupFunction<I1, I2, OUT> udf,
-			Keys.SelectorFunctionKeys<I1, K> key1, int[] key2, String name,
-			TypeInformation<OUT> type, TypeInformation<Tuple2<K, I1>> typeInfoWithKey1, TypeInformation<Tuple2<K, I2>> typeInfoWithKey2)
-	{
-		super(new TupleUnwrappingCoGrouper<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(typeInfoWithKey1,typeInfoWithKey2, type),
-				key1.computeLogicalKeyPositions(), new int[]{0}, name);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	public static final class TupleUnwrappingCoGrouper<I1, I2, OUT, K> extends WrappingFunction<CoGroupFunction<I1, I2, OUT>>
-		implements CoGroupFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>
-	{
-		private static final long serialVersionUID = 1L;
-		
-		private final TupleUnwrappingIterator<I1, K> iter1;
-		private final TupleUnwrappingIterator<I2, K> iter2;
-		
-		private TupleUnwrappingCoGrouper(CoGroupFunction<I1, I2, OUT> wrapped) {
-			super(wrapped);
-			
-			this.iter1 = new TupleUnwrappingIterator<I1, K>();
-			this.iter2 = new TupleUnwrappingIterator<I2, K>();
-		}
-
-
-		@Override
-		public void coGroup(Iterable<Tuple2<K, I1>> records1, Iterable<Tuple2<K, I2>> records2, Collector<OUT> out) throws Exception {
-			iter1.set(records1.iterator());
-			iter2.set(records2.iterator());
-			this.wrappedFunction.coGroup(iter1, iter2, out);
-		}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java
deleted file mode 100644
index efd52d5..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingJoinOperator.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.operators.translation;
-
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.operators.BinaryOperatorInformation;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.types.TypeInformation;
-import org.apache.flink.util.Collector;
-
-public class PlanUnwrappingJoinOperator<I1, I2, OUT, K> 
-	extends JoinOperatorBase<Tuple2<K, I1>, Tuple2<K, I2>, OUT, FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>>
-{
-
-	public PlanUnwrappingJoinOperator(FlatJoinFunction<I1, I2, OUT> udf,
-			Keys.SelectorFunctionKeys<I1, K> key1, Keys.SelectorFunctionKeys<I2, K> key2, String name,
-			TypeInformation<OUT> type, TypeInformation<Tuple2<K, I1>> typeInfoWithKey1, TypeInformation<Tuple2<K, I2>> typeInfoWithKey2)
-	{
-		super(new TupleUnwrappingJoiner<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(typeInfoWithKey1, typeInfoWithKey2, type),
-				key1.computeLogicalKeyPositions(), key2.computeLogicalKeyPositions(), name);
-	}
-	
-	public PlanUnwrappingJoinOperator(FlatJoinFunction<I1, I2, OUT> udf,
-			int[] key1, Keys.SelectorFunctionKeys<I2, K> key2, String name,
-			TypeInformation<OUT> type, TypeInformation<Tuple2<K, I1>> typeInfoWithKey1, TypeInformation<Tuple2<K, I2>> typeInfoWithKey2)
-	{
-		super(new TupleUnwrappingJoiner<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(typeInfoWithKey1, typeInfoWithKey2, type),
-				new int[]{0}, key2.computeLogicalKeyPositions(), name);
-	}
-	
-	public PlanUnwrappingJoinOperator(FlatJoinFunction<I1, I2, OUT> udf,
-			Keys.SelectorFunctionKeys<I1, K> key1, int[] key2, String name,
-			TypeInformation<OUT> type, TypeInformation<Tuple2<K, I1>> typeInfoWithKey1, TypeInformation<Tuple2<K, I2>> typeInfoWithKey2)
-	{
-		super(new TupleUnwrappingJoiner<I1, I2, OUT, K>(udf),
-				new BinaryOperatorInformation<Tuple2<K, I1>, Tuple2<K, I2>, OUT>(typeInfoWithKey1, typeInfoWithKey2, type),
-				key1.computeLogicalKeyPositions(), new int[]{0}, name);
-	}
-
-	public static final class TupleUnwrappingJoiner<I1, I2, OUT, K>
-		extends WrappingFunction<FlatJoinFunction<I1, I2, OUT>>
-		implements FlatJoinFunction<Tuple2<K, I1>, Tuple2<K, I2>, OUT>
-	{
-
-		private static final long serialVersionUID = 1L;
-		
-		private TupleUnwrappingJoiner(FlatJoinFunction<I1, I2, OUT> wrapped) {
-			super(wrapped);
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public void join (Tuple2<K, I1> value1, Tuple2<K, I2> value2, Collector<OUT> collector) throws Exception {
-			wrappedFunction.join ((I1)(value1.getField(1)), (I2)(value2.getField(1)), collector);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a32890ae/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java
deleted file mode 100644
index 97e67ca..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleKeyExtractingMapper.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.operators.translation;
-
-import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.types.NullFieldException;
-import org.apache.flink.types.NullKeyFieldException;
-
-
-public final class TupleKeyExtractingMapper<T, K> extends RichMapFunction<T, Tuple2<K, T>> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private final int pos;
-	
-	private final Tuple2<K, T> tuple = new Tuple2<K, T>();
-	
-	
-	public TupleKeyExtractingMapper(int pos) {
-		this.pos = pos;
-	}
-	
-	
-	@Override
-	public Tuple2<K, T> map(T value) throws Exception {
-		
-		Tuple v = (Tuple) value;
-		
-		try {
-			K key = v.getFieldNotNull(pos);
-			tuple.f0 = key;
-			tuple.f1 = value;
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-
-		return tuple;
-	}
-}


[21/60] git commit: Refactor TupleTypeInfo and add GenericPairComparator

Posted by al...@apache.org.
Refactor TupleTypeInfo and add GenericPairComparator

Now we have TupleTypeInfoBase, TupleSerializerBase, and TupleComparatorBase. They
are now super classes of TupleTypeInfo and the others.

Also rename compare on DataInputView to compareSerialized because Scala
cannot distinguish between the to compare methods for some reason.

This change is necessary for allowing the Scala API to reuse most of the
functionality.

The GenericPairComparator uses the new extractKeys method of
TypeComparator to compare values of any type. This replaces
TuplePairComparator and some other special-case pair comparators. This
is preparatory work for enabling support for Scala Tuples and POJO
comparators.


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

Branch: refs/heads/master
Commit: 57b8e66ae9a34534efac194d794b11d44914f711
Parents: 7f946ce
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Aug 22 12:24:11 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../api/common/typeutils/TypeComparator.java    |  18 +-
 .../typeutils/base/BasicTypeComparator.java     |  17 ++
 .../typeutils/base/BooleanComparator.java       |   2 +-
 .../common/typeutils/base/ByteComparator.java   |   2 +-
 .../common/typeutils/base/CharComparator.java   |   2 +-
 .../common/typeutils/base/DoubleComparator.java |   2 +-
 .../common/typeutils/base/FloatComparator.java  |   2 +-
 .../common/typeutils/base/IntComparator.java    |   2 +-
 .../common/typeutils/base/LongComparator.java   |   2 +-
 .../common/typeutils/base/ShortComparator.java  |   2 +-
 .../common/typeutils/base/StringComparator.java |   2 +-
 .../common/typeutils/ComparatorTestBase.java    |  12 +-
 .../flink/api/java/typeutils/TupleTypeInfo.java | 137 +--------
 .../api/java/typeutils/TupleTypeInfoBase.java   | 117 ++++++++
 .../runtime/CopyableValueComparator.java        |  20 +-
 .../runtime/GenericPairComparator.java          |  91 ++++++
 .../runtime/GenericTypeComparator.java          |  17 +-
 .../java/typeutils/runtime/PojoComparator.java  |  26 +-
 .../typeutils/runtime/PojoPairComparator.java   | 153 ----------
 .../runtime/RuntimePairComparatorFactory.java   | 121 +-------
 .../java/typeutils/runtime/TupleComparator.java | 257 ++---------------
 .../typeutils/runtime/TupleComparatorBase.java  | 281 +++++++++++++++++++
 .../runtime/TupleLeadingFieldComparator.java    | 155 ----------
 .../TupleLeadingFieldPairComparator.java        |  70 -----
 .../typeutils/runtime/TuplePairComparator.java  | 105 -------
 .../java/typeutils/runtime/TupleSerializer.java |  73 +----
 .../typeutils/runtime/TupleSerializerBase.java  | 100 +++++++
 .../java/typeutils/runtime/ValueComparator.java |  20 +-
 .../typeutils/runtime/WritableComparator.java   |  18 +-
 .../runtime/record/RecordComparator.java        |  14 +-
 .../api/java/typeutils/TypeInfoParserTest.java  |   6 +-
 .../runtime/GenericPairComparatorTest.java      |  89 ++++++
 .../TupleLeadingFieldComparatorTest.java        |  74 -----
 .../TupleLeadingFieldPairComparatorTest.java    |  66 -----
 .../runtime/TuplePairComparatorTest.java        |  78 -----
 .../operators/sort/NormalizedKeySorter.java     |   2 +-
 .../testutils/types/IntListComparator.java      |  18 +-
 .../testutils/types/IntPairComparator.java      |  18 +-
 .../testutils/types/StringPairComparator.java   |  17 +-
 .../operators/util/OutputEmitterTest.java       |  18 +-
 .../VertexWithAdjacencyListComparator.java      |  20 +-
 .../VertexWithRankAndDanglingComparator.java    |  20 +-
 .../types/VertexWithRankComparator.java         |  20 +-
 43 files changed, 992 insertions(+), 1294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
index ae9e4f1..1958824 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
@@ -177,7 +177,7 @@ public abstract class TypeComparator<T> implements Serializable {
 	 * 
 	 *  @see java.util.Comparator#compare(Object, Object)
 	 */
-	public abstract int compare(DataInputView firstSource, DataInputView secondSource) throws IOException;
+	public abstract int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException;
 	
 	// --------------------------------------------------------------------------------------------
 	
@@ -286,7 +286,21 @@ public abstract class TypeComparator<T> implements Serializable {
 	public abstract TypeComparator<T> duplicate();
 	
 	// --------------------------------------------------------------------------------------------
-	
+
+	/**
+	 * Extracts the key fields from a record. This is for use by the PairComparator to provide
+	 * interoperability between different record types.
+	 */
+	public abstract Object[] extractKeys(T record);
+
+	/**
+	 * Get the field comparators. This is used together with {@link #extractKeys(Object)} to provide
+	 * interoperability between different record types.
+	 */
+	public abstract TypeComparator[] getComparators();
+
+	// --------------------------------------------------------------------------------------------
+
 	@SuppressWarnings("rawtypes")
 	public int compareAgainstReference(Comparable[] keys) {
 		throw new UnsupportedOperationException("Workaround hack.");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BasicTypeComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BasicTypeComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BasicTypeComparator.java
index 8c6158c..a57bc62 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BasicTypeComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BasicTypeComparator.java
@@ -32,6 +32,12 @@ public abstract class BasicTypeComparator<T extends Comparable<T>> extends TypeC
 	private transient T reference;
 	
 	protected final boolean ascendingComparison;
+
+	// This is used in extractKeys, so that we don't create a new array for every call.
+	private final Comparable[] extractedKey = new Comparable[1];
+
+	// For use by getComparators
+	private final TypeComparator[] comparators = new TypeComparator[] {this};
 	
 
 	protected BasicTypeComparator(boolean ascending) {
@@ -81,6 +87,17 @@ public abstract class BasicTypeComparator<T extends Comparable<T>> extends TypeC
 	}
 
 	@Override
+	public Object[] extractKeys(T record) {
+		extractedKey[0] = record;
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
+
+	@Override
 	public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
 		throw new UnsupportedOperationException();
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanComparator.java
index 2dd1aa2..25d04b1 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanComparator.java
@@ -35,7 +35,7 @@ public final class BooleanComparator extends BasicTypeComparator<Boolean> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		final int fs = firstSource.readBoolean() ? 1 : 0;
 		final int ss = secondSource.readBoolean() ? 1 : 0;
 		int comp = fs - ss; 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteComparator.java
index 9d147c4..a301cc2 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteComparator.java
@@ -35,7 +35,7 @@ public final class ByteComparator extends BasicTypeComparator<Byte> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		byte b1 = firstSource.readByte();
 		byte b2 = secondSource.readByte();
 		int comp = (b1 < b2 ? -1 : (b1 == b2 ? 0 : 1)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharComparator.java
index 45f3d06..eff0260 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharComparator.java
@@ -35,7 +35,7 @@ public final class CharComparator extends BasicTypeComparator<Character> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		char c1 = firstSource.readChar();
 		char c2 = secondSource.readChar();
 		int comp = (c1 < c2 ? -1 : (c1 == c2 ? 0 : 1)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleComparator.java
index 39f13bb..3d89760 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleComparator.java
@@ -35,7 +35,7 @@ public final class DoubleComparator extends BasicTypeComparator<Double> {
 	}
 
 	@Override 
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException { 
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		double l1 = firstSource.readDouble(); 
 		double l2 = secondSource.readDouble(); 
 		int comp = (l1 < l2 ? -1 : (l1 > l2 ? 1 : 0)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatComparator.java
index 3ca7d0d..7f0d971 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatComparator.java
@@ -35,7 +35,7 @@ public final class FloatComparator extends BasicTypeComparator<Float> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		float l1 = firstSource.readFloat();
 		float l2 = secondSource.readFloat();
 		int comp = (l1 < l2 ? -1 : (l1 > l2 ? 1 : 0)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntComparator.java
index 7e22d65..9622fb2 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntComparator.java
@@ -35,7 +35,7 @@ public final class IntComparator extends BasicTypeComparator<Integer> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		int i1 = firstSource.readInt();
 		int i2 = secondSource.readInt();
 		int comp = (i1 < i2 ? -1 : (i1 == i2 ? 0 : 1)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongComparator.java
index 28d80b8..2bb0b34 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongComparator.java
@@ -35,7 +35,7 @@ public final class LongComparator extends BasicTypeComparator<Long> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		long l1 = firstSource.readLong();
 		long l2 = secondSource.readLong();
 		int comp = (l1 < l2 ? -1 : (l1 == l2 ? 0 : 1)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortComparator.java
index efec90b..03707ad 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortComparator.java
@@ -35,7 +35,7 @@ public final class ShortComparator extends BasicTypeComparator<Short> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		short s1 = firstSource.readShort();
 		short s2 = secondSource.readShort();
 		int comp = (s1 < s2 ? -1 : (s1 == s2 ? 0 : 1)); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringComparator.java
index dd706db..5bcb82a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringComparator.java
@@ -42,7 +42,7 @@ public final class StringComparator extends BasicTypeComparator<String> {
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		String s1 = StringValue.readString(firstSource);
 		String s2 = StringValue.readString(secondSource);
 		int comp = s1.compareTo(s2); 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
index cd952d6..5392fa9 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
@@ -26,8 +26,6 @@ import java.io.IOException;
 
 import static org.junit.Assert.*;
 
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
@@ -108,7 +106,7 @@ public abstract class ComparatorTestBase<T> {
 				writeSortedData(d, out1);
 				in1 = out1.getInputView();
 
-				assertTrue(comparator.compare(in1, in2) == 0);
+				assertTrue(comparator.compareSerialized(in1, in2) == 0);
 			}
 		} catch (Exception e) {
 			System.err.println(e.getMessage());
@@ -173,16 +171,16 @@ public abstract class ComparatorTestBase<T> {
 					in2 = out2.getInputView();
 
 					if (greater && ascending) {
-						assertTrue(comparator.compare(in1, in2) < 0);
+						assertTrue(comparator.compareSerialized(in1, in2) < 0);
 					}
 					if (greater && !ascending) {
-						assertTrue(comparator.compare(in1, in2) > 0);
+						assertTrue(comparator.compareSerialized(in1, in2) > 0);
 					}
 					if (!greater && ascending) {
-						assertTrue(comparator.compare(in2, in1) > 0);
+						assertTrue(comparator.compareSerialized(in2, in1) > 0);
 					}
 					if (!greater && !ascending) {
-						assertTrue(comparator.compare(in2, in1) < 0);
+						assertTrue(comparator.compareSerialized(in2, in1) < 0);
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
index b9dce11..25be7f1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
@@ -23,81 +23,29 @@ import java.util.Arrays;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
-import org.apache.flink.api.java.typeutils.runtime.TupleLeadingFieldComparator;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.types.TypeInformation;
 
-
 //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator
 import org.apache.flink.api.java.tuple.*;
 //CHECKSTYLE.ON: AvoidStarImport
 
 
-public class TupleTypeInfo<T extends Tuple> extends TypeInformation<T> implements CompositeType<T> {
-	
-	private final TypeInformation<?>[] types;
-	
-	private final Class<T> tupleType;
-	
-	
-	public TupleTypeInfo(TypeInformation<?>... types) {
-		if (types == null || types.length == 0 || types.length > Tuple.MAX_ARITY) {
-			throw new IllegalArgumentException();
-		}
 
-		@SuppressWarnings("unchecked")
-		Class<T> typeClass = (Class<T>) CLASSES[types.length - 1];
-		
-		this.types = types;
-		this.tupleType = typeClass;
+public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
+
+	@SuppressWarnings("unchecked")
+	public TupleTypeInfo(TypeInformation<?>... types) {
+		this((Class<T>) CLASSES[types.length - 1], types);
 	}
-	
+
 	public TupleTypeInfo(Class<T> tupleType, TypeInformation<?>... types) {
+		super(tupleType, types);
 		if (types == null || types.length == 0 || types.length > Tuple.MAX_ARITY) {
 			throw new IllegalArgumentException();
 		}
-		
-		this.tupleType = tupleType;
-		this.types = types;
-	}
-	
-	
-	@Override
-	public boolean isBasicType() {
-		return false;
-	}
-
-	@Override
-	public boolean isTupleType() {
-		return true;
-	}
-
-	@Override
-	public int getArity() {
-		return types.length;
-	}
-
-	@Override
-	public Class<T> getTypeClass() {
-		return tupleType;
 	}
 
-	
-	public <X> TypeInformation<X> getTypeAt(int pos) {
-		if (pos < 0 || pos >= this.types.length) {
-			throw new IndexOutOfBoundsException();
-		}
-
-		@SuppressWarnings("unchecked")
-		TypeInformation<X> typed = (TypeInformation<X>) this.types[pos];
-		return typed;
-	}
-	
-	@Override
-	public boolean isKeyType() {
-		return this.isValidKeyType(this);
-	}
-	
 	@Override
 	public TupleSerializer<T> createSerializer() {
 		TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[getArity()];
@@ -118,14 +66,7 @@ public class TupleTypeInfo<T extends Tuple> extends TypeInformation<T> implement
 		{
 			throw new IllegalArgumentException();
 		}
-		
-		// special case for tuples where field zero is the key field
-		if (logicalKeyFields.length == 1 && logicalKeyFields[0] == 0 && !types[0].isTupleType()) {
-			return createLeadingFieldComparator(orders[0], types[0]);
-		}
-		
-		// --- general case ---
-		
+
 		int maxKey = -1;
 		for (int key : logicalKeyFields){
 			maxKey = Math.max(key, maxKey);
@@ -169,39 +110,12 @@ public class TupleTypeInfo<T extends Tuple> extends TypeInformation<T> implement
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof TupleTypeInfo) {
-			@SuppressWarnings("unchecked")
-			TupleTypeInfo<T> other = (TupleTypeInfo<T>) obj;
-			return ((this.tupleType == null && other.tupleType == null) || this.tupleType.equals(other.tupleType)) &&
-					Arrays.deepEquals(this.types, other.types);
-			
-		} else {
-			return false;
-		}
-	}
-	
-	@Override
-	public int hashCode() {
-		return this.types.hashCode() ^ Arrays.deepHashCode(this.types);
-	}
-	
+
 	@Override
 	public String toString() {
-		StringBuilder bld = new StringBuilder("Tuple");
-		bld.append(types.length).append('<');
-		bld.append(types[0]);
-		
-		for (int i = 1; i < types.length; i++) {
-			bld.append(", ").append(types[i]);
-		}
-		
-		bld.append('>');
-		return bld.toString();
+		return "Java " + super.toString();
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 	
 	public static <X extends Tuple> TupleTypeInfo<X> getBasicTupleTypeInfo(Class<?>... basicTypes) {
@@ -227,23 +141,7 @@ public class TupleTypeInfo<T extends Tuple> extends TypeInformation<T> implement
 		TupleTypeInfo<X> tupleInfo = (TupleTypeInfo<X>) new TupleTypeInfo<Tuple>(infos);
 		return tupleInfo;
 	}
-	
-	private boolean isValidKeyType(TypeInformation<?> typeInfo) {
-		if(typeInfo instanceof TupleTypeInfo) {
-			TupleTypeInfo<?> tupleType = ((TupleTypeInfo<?>)typeInfo);
-			for(int i=0;i<tupleType.getArity();i++) {
-				if (!isValidKeyType(tupleType.getTypeAt(i))) {
-					return false;
-				}
-			}
-			return true;
-		} else if(typeInfo.isKeyType()) {
-				return true;
-		} else {
-			return false;
-		}
-	}
-	
+
 	// --------------------------------------------------------------------------------------------	
 	// The following lines are generated.
 	// --------------------------------------------------------------------------------------------
@@ -254,15 +152,4 @@ public class TupleTypeInfo<T extends Tuple> extends TypeInformation<T> implement
 		Tuple1.class, Tuple2.class, Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class, Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class, Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class, Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class, Tuple24.class, Tuple25.class
 	};
 	// END_OF_TUPLE_DEPENDENT_CODE
-	
-	
-	private static final <T extends Tuple, K> TypeComparator<T> createLeadingFieldComparator(boolean ascending, TypeInformation<?> info) {
-		if (!(info.isKeyType() && info instanceof AtomicType)) {
-			throw new IllegalArgumentException("The field at position 0 (" + info + ") is no atomic key type.");
-		}
-		
-		@SuppressWarnings("unchecked")
-		AtomicType<K> typedInfo = (AtomicType<K>) info;
-		return new TupleLeadingFieldComparator<T, K>(typedInfo.createComparator(ascending));
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
new file mode 100644
index 0000000..bca29dc
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils;
+
+import java.util.Arrays;
+
+import org.apache.flink.types.TypeInformation;
+
+public abstract class TupleTypeInfoBase<T> extends TypeInformation<T> implements CompositeType<T> {
+	
+	protected final TypeInformation<?>[] types;
+	
+	protected final Class<T> tupleType;
+	
+	public TupleTypeInfoBase(Class<T> tupleType, TypeInformation<?>... types) {
+		this.tupleType = tupleType;
+		this.types = types;
+	}
+
+	@Override
+	public boolean isBasicType() {
+		return false;
+	}
+
+	@Override
+	public boolean isTupleType() {
+		return true;
+	}
+
+	@Override
+	public int getArity() {
+		return types.length;
+	}
+
+	@Override
+	public Class<T> getTypeClass() {
+		return tupleType;
+	}
+
+	
+	public <X> TypeInformation<X> getTypeAt(int pos) {
+		if (pos < 0 || pos >= this.types.length) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		@SuppressWarnings("unchecked")
+		TypeInformation<X> typed = (TypeInformation<X>) this.types[pos];
+		return typed;
+	}
+	
+	@Override
+	public boolean isKeyType() {
+		return isValidKeyType(this);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof TupleTypeInfoBase) {
+			@SuppressWarnings("unchecked")
+			TupleTypeInfoBase<T> other = (TupleTypeInfoBase<T>) obj;
+			return ((this.tupleType == null && other.tupleType == null) || this.tupleType.equals(other.tupleType)) &&
+					Arrays.deepEquals(this.types, other.types);
+			
+		} else {
+			return false;
+		}
+	}
+	
+	@Override
+	public int hashCode() {
+		return this.types.hashCode() ^ Arrays.deepHashCode(this.types);
+	}
+
+	private boolean isValidKeyType(TypeInformation<?> typeInfo) {
+		if(typeInfo instanceof TupleTypeInfoBase) {
+			TupleTypeInfoBase<?> tupleType = ((TupleTypeInfoBase<?>)typeInfo);
+			for(int i=0;i<tupleType.getArity();i++) {
+				if (!isValidKeyType(tupleType.getTypeAt(i))) {
+					return false;
+				}
+			}
+			return true;
+		} else  {
+			return typeInfo.isKeyType();
+		}
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder bld = new StringBuilder("Tuple");
+		bld.append(types.length).append('<');
+		bld.append(types[0]);
+		
+		for (int i = 1; i < types.length; i++) {
+			bld.append(", ").append(types[i]);
+		}
+		
+		bld.append('>');
+		return bld.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
index cf692ac..6911580 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java
@@ -42,8 +42,11 @@ public class CopyableValueComparator<T extends CopyableValue<T> & Comparable<T>>
 	private transient T reference;
 	
 	private transient T tempReference;
-	
-	
+
+	private final Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {this};
+
 	public CopyableValueComparator(boolean ascending, Class<T> type) {
 		this.type = type;
 		this.ascendingComparison = ascending;
@@ -79,7 +82,7 @@ public class CopyableValueComparator<T extends CopyableValue<T> & Comparable<T>>
 	}
 	
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		if (tempReference == null) {
 			tempReference = InstantiationUtil.instantiate(type, CopyableValue.class);
 		}
@@ -121,6 +124,17 @@ public class CopyableValueComparator<T extends CopyableValue<T> & Comparable<T>>
 	public TypeComparator<T> duplicate() {
 		return new CopyableValueComparator<T>(ascendingComparison, type);
 	}
+
+	@Override
+	public Object[] extractKeys(T record) {
+		extractedKey[0] = record;
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	// unsupported normalization

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparator.java
new file mode 100644
index 0000000..d47c1ef
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparator.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+
+
+public class GenericPairComparator<T1, T2> extends TypePairComparator<T1, T2>
+		implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final TypeComparator<T1> comparator1;
+	private final TypeComparator<T2> comparator2;
+
+	private final TypeComparator<Object>[] comparators1;
+	private final TypeComparator<Object>[] comparators2;
+
+	private final Object[] referenceKeyFields;
+
+	@SuppressWarnings("unchecked")
+	public GenericPairComparator(TypeComparator<T1> comparator1, TypeComparator<T2> comparator2) {
+		this.comparator1 = comparator1;
+		this.comparator2 = comparator2;
+		this.comparators1 = comparator1.getComparators();
+		this.comparators2 = comparator2.getComparators();
+
+		if(comparators1.length != comparators2.length) {
+			throw new IllegalArgumentException("Number of key fields and comparators differ.");
+		}
+
+		int numKeys = comparators1.length;
+		
+		for(int i = 0; i < numKeys; i++) {
+			this.comparators1[i] = comparators1[i].duplicate();
+			this.comparators2[i] = comparators2[i].duplicate();
+		}
+
+		this.referenceKeyFields = new Object[numKeys];
+	}
+	
+	@Override
+	public void setReference(T1 reference) {
+		Object[] keys = comparator1.extractKeys(reference);
+		System.arraycopy(keys, 0, referenceKeyFields, 0, keys.length);
+	}
+
+	@Override
+	public boolean equalToReference(T2 candidate) {
+		Object[] keys = comparator2.extractKeys(candidate);
+		for (int i = 0; i < this.comparators1.length; i++) {
+			if (this.comparators1[i].compare(referenceKeyFields[i], keys[i]) != 0) {
+				return false;
+			}
+		}
+		return true;
+	}
+
+	@Override
+	public int compareToReference(T2 candidate) {
+		Object[] keys = comparator2.extractKeys(candidate);
+		for (int i = 0; i < this.comparators1.length; i++) {
+			// We reverse ordering here because our "compareToReference" does work in a mirrored
+			// way compared to Comparable.compareTo
+			int res = this.comparators1[i].compare(keys[i], referenceKeyFields[i]);
+			if(res != 0) {
+				return res;
+			}
+		}
+		return 0;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
index d78b491..602b1cb 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java
@@ -52,6 +52,10 @@ public class GenericTypeComparator<T extends Comparable<T>> extends TypeComparat
 
 	private transient Kryo kryo;
 
+	private final Comparable[] extractedKey = new Comparable[1];
+
+	private final TypeComparator[] comparators = new TypeComparator[] {this};
+
 	// ------------------------------------------------------------------------
 
 	public GenericTypeComparator(boolean ascending, TypeSerializer<T> serializer, Class<T> type) {
@@ -100,7 +104,7 @@ public class GenericTypeComparator<T extends Comparable<T>> extends TypeComparat
 	}
 
 	@Override
-	public int compare(final DataInputView firstSource, final DataInputView secondSource) throws IOException {
+	public int compareSerialized(final DataInputView firstSource, final DataInputView secondSource) throws IOException {
 		if (this.serializer == null) {
 			this.serializer = this.serializerFactory.getSerializer();
 		}
@@ -164,6 +168,17 @@ public class GenericTypeComparator<T extends Comparable<T>> extends TypeComparat
 		}
 	}
 
+	@Override
+	public Object[] extractKeys(T record) {
+		extractedKey[0] = record;
+		return extractedKey;
+	}
+
+	@Override
+	public TypeComparator[] getComparators() {
+		return comparators;
+	}
+
 	// ------------------------------------------------------------------------
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
index 41f73c4..a61e9ef 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java
@@ -52,6 +52,7 @@ public final class PojoComparator<T> extends TypeComparator<T> implements java.i
 
 	private final Class<T> type;
 
+	private final Comparable[] extractedKeys;
 
 	@SuppressWarnings("unchecked")
 	public PojoComparator(Field[] keyFields, TypeComparator<?>[] comparators, TypeSerializer<T> serializer, Class<T> type) {
@@ -101,6 +102,8 @@ public final class PojoComparator<T> extends TypeComparator<T> implements java.i
 		this.numLeadingNormalizableKeys = nKeys;
 		this.normalizableKeyPrefixLen = nKeyLen;
 		this.invertNormKey = inverted;
+
+		extractedKeys = new Comparable[keyFields.length];
 	}
 
 	@SuppressWarnings("unchecked")
@@ -128,7 +131,7 @@ public final class PojoComparator<T> extends TypeComparator<T> implements java.i
 			throw new RuntimeException("Cannot copy serializer", e);
 		}
 
-
+		extractedKeys = new Comparable[keyFields.length];
 	}
 
 	private void writeObject(ObjectOutputStream out)
@@ -172,7 +175,7 @@ public final class PojoComparator<T> extends TypeComparator<T> implements java.i
 		return this.keyFields;
 	}
 
-	public TypeComparator<Object>[] getComparators() {
+	public TypeComparator[] getComparators() {
 		return this.comparators;
 	}
 
@@ -269,7 +272,7 @@ public final class PojoComparator<T> extends TypeComparator<T> implements java.i
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
 		T first = this.serializer.createInstance();
 		T second = this.serializer.createInstance();
 
@@ -343,6 +346,23 @@ public final class PojoComparator<T> extends TypeComparator<T> implements java.i
 		return new PojoComparator<T>(this);
 	}
 
+	@Override
+	public Object[] extractKeys(T record) {
+		int i = 0;
+		try {
+			for (; i < keyFields.length; i++) {
+				extractedKeys[i] = (Comparable) keyFields[i].get(record);
+			}
+		}
+		catch (IllegalAccessException iaex) {
+			throw new RuntimeException("This should not happen since we call setAccesssible(true) in PojoTypeInfo.");
+		}
+		catch (NullPointerException npex) {
+			throw new NullKeyFieldException(this.keyFields[i].toString());
+		}
+		return extractedKeys;
+	}
+
 	// --------------------------------------------------------------------------------------------
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoPairComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoPairComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoPairComparator.java
deleted file mode 100644
index 29266be..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoPairComparator.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.lang.reflect.Field;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-
-
-public class PojoPairComparator<T1, T2> extends TypePairComparator<T1, T2> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	@SuppressWarnings("unused")
-	private final int[] keyPositions1, keyPositions2;
-	private transient Field[] keyFields1, keyFields2;
-	private final TypeComparator<Object>[] comparators1;
-	private final TypeComparator<Object>[] comparators2;
-
-	@SuppressWarnings("unchecked")
-	public PojoPairComparator(int[] keyPositions1, Field[] keyFields1, int[] keyPositions2, Field[] keyFields2, TypeComparator<Object>[] comparators1, TypeComparator<Object>[] comparators2) {
-
-		if(keyPositions1.length != keyPositions2.length
-			|| keyPositions1.length != comparators1.length
-			|| keyPositions2.length != comparators2.length) {
-
-			throw new IllegalArgumentException("Number of key fields and comparators differ.");
-		}
-
-		int numKeys = keyPositions1.length;
-
-		this.keyPositions1 = keyPositions1;
-		this.keyPositions2 = keyPositions2;
-		this.keyFields1 = keyFields1;
-		this.keyFields2 = keyFields2;
-		this.comparators1 = new TypeComparator[numKeys];
-		this.comparators2 = new TypeComparator[numKeys];
-
-		for(int i = 0; i < numKeys; i++) {
-			this.comparators1[i] = comparators1[i].duplicate();
-			this.comparators2[i] = comparators2[i].duplicate();
-		}
-	}
-
-	private void writeObject(ObjectOutputStream out)
-			throws IOException, ClassNotFoundException {
-		out.defaultWriteObject();
-		out.writeInt(keyFields1.length);
-		for (Field field: keyFields1) {
-			out.writeObject(field.getDeclaringClass());
-			out.writeUTF(field.getName());
-		}
-		out.writeInt(keyFields2.length);
-		for (Field field: keyFields2) {
-			out.writeObject(field.getDeclaringClass());
-			out.writeUTF(field.getName());
-		}
-	}
-
-	private void readObject(ObjectInputStream in)
-			throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-		int numKeyFields = in.readInt();
-		keyFields1 = new Field[numKeyFields];
-		for (int i = 0; i < numKeyFields; i++) {
-			Class<?> clazz = (Class<?>)in.readObject();
-			String fieldName = in.readUTF();
-			try {
-				keyFields1[i] = clazz.getField(fieldName);
-				keyFields1[i].setAccessible(true);
-			} catch (NoSuchFieldException e) {
-				throw new RuntimeException("Class resolved at TaskManager is not compatible with class read during Plan setup.");
-			}
-		}
-		numKeyFields = in.readInt();
-		keyFields2 = new Field[numKeyFields];
-		for (int i = 0; i < numKeyFields; i++) {
-			Class<?> clazz = (Class<?>)in.readObject();
-			String fieldName = in.readUTF();
-			try {
-				keyFields2[i] = clazz.getField(fieldName);
-				keyFields2[i].setAccessible(true);
-			} catch (NoSuchFieldException e) {
-				throw new RuntimeException("Class resolved at TaskManager is not compatible with class read during Plan setup.");
-			}
-		}
-	}
-
-	@Override
-	public void setReference(T1 reference) {
-		for(int i=0; i < this.comparators1.length; i++) {
-			try {
-				this.comparators1[i].setReference(keyFields1[i].get(reference));
-			} catch (IllegalAccessException e) {
-				throw new RuntimeException("This should not happen since we call setAccesssible(true) in PojoTypeInfo.");
-			}
-		}
-	}
-
-	@Override
-	public boolean equalToReference(T2 candidate) {
-		for(int i=0; i < this.comparators1.length; i++) {
-			try {
-				if(!this.comparators1[i].equalToReference(keyFields2[i].get(candidate))) {
-					return false;
-				}
-			} catch (IllegalAccessException e) {
-				throw new RuntimeException("Class resolved at TaskManager is not compatible with class read during Plan setup.");
-			}
-		}
-		return true;
-	}
-
-	@Override
-	public int compareToReference(T2 candidate) {
-		for(int i=0; i < this.comparators1.length; i++) {
-			try {
-				this.comparators2[i].setReference(keyFields2[i].get(candidate));
-			} catch (IllegalAccessException e) {
-				throw new RuntimeException("Class resolved at TaskManager is not compatible with class read during Plan setup.");
-			}
-			int res = this.comparators1[i].compareToReference(this.comparators2[i]);
-			if(res != 0) {
-				return res;
-			}
-		}
-		return 0;
-	}
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
index f210cdb..17c939c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java
@@ -16,129 +16,30 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.java.typeutils.runtime;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-import org.apache.flink.api.java.tuple.Tuple;
-
 
-public final class RuntimePairComparatorFactory<T1 extends Tuple, T2 extends Tuple> implements TypePairComparatorFactory<T1, T2>, java.io.Serializable {
+public final class RuntimePairComparatorFactory<T1, T2>
+		implements TypePairComparatorFactory<T1, T2>, java.io.Serializable {
 
 	private static final long serialVersionUID = 1L;
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public TypePairComparator<T1, T2> createComparator12(TypeComparator<T1> comparator1, TypeComparator<T2> comparator2) {
-
-		if ((comparator1 instanceof TupleLeadingFieldComparator) && (comparator2 instanceof TupleLeadingFieldComparator)) {
-
-			TypeComparator<?> comp1 = ((TupleLeadingFieldComparator<?,?>) comparator1).getFieldComparator();
-			TypeComparator<?> comp2 = ((TupleLeadingFieldComparator<?,?>) comparator2).getFieldComparator();
-
-			return createLeadingFieldPairComp(comp1, comp2);
-		}
-		else {
-			int[] keyPos1;
-			int[] keyPos2;
-			TypeComparator<Object>[] comps1;
-			TypeComparator<Object>[] comps2;
-			
-			// get info from first comparator
-			if (comparator1 instanceof TupleComparator) {
-				TupleComparator<?> tupleComp1 = (TupleComparator<?>) comparator1;
-				keyPos1 = tupleComp1.getKeyPositions();
-				comps1 = tupleComp1.getComparators();
-			}
-			else if (comparator1 instanceof TupleLeadingFieldComparator) {
-				TupleLeadingFieldComparator<?, ?> tupleComp1 = (TupleLeadingFieldComparator<?, ?>) comparator1;
-				keyPos1 = new int[] {0};
-				comps1 = new TypeComparator[] { tupleComp1.getFieldComparator() };
-			}
-			else {
-				throw new IllegalArgumentException("Cannot instantiate pair comparator from the given comparator: " + comparator1);
-			}
-			
-			// get info from second comparator
-			if (comparator2 instanceof TupleComparator) {
-				TupleComparator<?> tupleComp2 = (TupleComparator<?>) comparator2;
-				keyPos2 = tupleComp2.getKeyPositions();
-				comps2 = tupleComp2.getComparators();
-			}
-			else if (comparator2 instanceof TupleLeadingFieldComparator) {
-				TupleLeadingFieldComparator<?, ?> tupleComp2 = (TupleLeadingFieldComparator<?, ?>) comparator2;
-				keyPos2 = new int[] {0};
-				comps2 = new TypeComparator[] { tupleComp2.getFieldComparator() };
-			}
-			else {
-				throw new IllegalArgumentException("Cannot instantiate pair comparator from the given comparator: " + comparator1);
-			}
-
-			return (TypePairComparator<T1, T2>) new TuplePairComparator<Tuple, Tuple>(keyPos1, keyPos2, comps1, comps2);
-		}
+	public TypePairComparator<T1, T2> createComparator12(
+			TypeComparator<T1> comparator1,
+			TypeComparator<T2> comparator2) {
+		return new GenericPairComparator<T1, T2>(comparator1, comparator2);
 	}
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public TypePairComparator<T2, T1> createComparator21(TypeComparator<T1> comparator1, TypeComparator<T2> comparator2) {
-		
-		if ((comparator1 instanceof TupleLeadingFieldComparator) && (comparator2 instanceof TupleLeadingFieldComparator)) {
-
-			TypeComparator<?> comp1 = ((TupleLeadingFieldComparator<?,?>) comparator1).getFieldComparator();
-			TypeComparator<?> comp2 = ((TupleLeadingFieldComparator<?,?>) comparator2).getFieldComparator();
-
-			return createLeadingFieldPairComp(comp2, comp1);
-		}
-		else {
-			int[] keyPos1;
-			int[] keyPos2;
-			TypeComparator<Object>[] comps1;
-			TypeComparator<Object>[] comps2;
-			
-			// get info from first comparator
-			if (comparator1 instanceof TupleComparator) {
-				TupleComparator<?> tupleComp1 = (TupleComparator<?>) comparator1;
-				keyPos1 = tupleComp1.getKeyPositions();
-				comps1 = tupleComp1.getComparators();
-			}
-			else if (comparator1 instanceof TupleLeadingFieldComparator) {
-				TupleLeadingFieldComparator<?, ?> tupleComp1 = (TupleLeadingFieldComparator<?, ?>) comparator1;
-				keyPos1 = new int[] {0};
-				comps1 = new TypeComparator[] { tupleComp1.getFieldComparator() };
-			}
-			else {
-				throw new IllegalArgumentException("Cannot instantiate pair comparator from the given comparator: " + comparator1);
-			}
-			
-			// get info from second comparator
-			if (comparator2 instanceof TupleComparator) {
-				TupleComparator<?> tupleComp2 = (TupleComparator<?>) comparator2;
-				keyPos2 = tupleComp2.getKeyPositions();
-				comps2 = tupleComp2.getComparators();
-			}
-			else if (comparator2 instanceof TupleLeadingFieldComparator) {
-				TupleLeadingFieldComparator<?, ?> tupleComp2 = (TupleLeadingFieldComparator<?, ?>) comparator2;
-				keyPos2 = new int[] {0};
-				comps2 = new TypeComparator[] { tupleComp2.getFieldComparator() };
-			}
-			else {
-				throw new IllegalArgumentException("Cannot instantiate pair comparator from the given comparator: " + comparator1);
-			}
-
-			return (TypePairComparator<T2, T1>) new TuplePairComparator<Tuple, Tuple>(keyPos2, keyPos1, comps2, comps1);
-		}
-	}
-	
-	private static <K, T1 extends Tuple, T2 extends Tuple> TupleLeadingFieldPairComparator<K, T1, T2> createLeadingFieldPairComp(
-			TypeComparator<?> comp1, TypeComparator<?> comp2)
-	{
-		@SuppressWarnings("unchecked")
-		TypeComparator<K> c1 = (TypeComparator<K>) comp1;
-		@SuppressWarnings("unchecked")
-		TypeComparator<K> c2 = (TypeComparator<K>) comp2;
-		
-		return new TupleLeadingFieldPairComparator<K, T1, T2>(c1, c2);
+	public TypePairComparator<T2, T1> createComparator21(
+			TypeComparator<T1> comparator1,
+			TypeComparator<T2> comparator2) {
+		return new GenericPairComparator<T2, T1>(comparator2, comparator1);
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
index 9de9824..7b3976f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
@@ -18,135 +18,33 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
-import java.io.IOException;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.types.KeyFieldOutOfBoundsException;
 import org.apache.flink.types.NullFieldException;
 import org.apache.flink.types.NullKeyFieldException;
 
 
-public final class TupleComparator<T extends Tuple> extends TypeComparator<T> implements java.io.Serializable {
+public final class TupleComparator<T extends Tuple> extends TupleComparatorBase<T> {
 
 	private static final long serialVersionUID = 1L;
 
-	/** key positions describe which fields are keys in what order */
-	private final int[] keyPositions;
-	
-	/** comparators for the key fields, in the same order as the key fields */
-	private final TypeComparator<Object>[] comparators;
+	private final Object[] extractedKeys;
 
-	/** serializer factories to duplicate non thread-safe serializers */
-	private final TypeSerializerFactory<Object>[] serializerFactories;
-	
-	
-	private final int[] normalizedKeyLengths;
-	
-	private final int numLeadingNormalizableKeys;
-	
-	private final int normalizableKeyPrefixLen;
-	
-	private final boolean invertNormKey;
-	
-	
-	/** serializers to deserialize the first n fields for comparison */
-	private transient TypeSerializer<Object>[] serializers;
-	
-	// cache for the deserialized field objects
-	private transient Object[] deserializedFields1;
-	private transient Object[] deserializedFields2;
-	
-	
 	@SuppressWarnings("unchecked")
 	public TupleComparator(int[] keyPositions, TypeComparator<?>[] comparators, TypeSerializer<?>[] serializers) {
-		// set the default utils
-		this.keyPositions = keyPositions;
-		this.comparators = (TypeComparator<Object>[]) comparators;
-		this.serializers = (TypeSerializer<Object>[]) serializers;
-	
-		// set the serializer factories.
-		this.serializerFactories = new TypeSerializerFactory[this.serializers.length];
-		for (int i = 0; i < serializers.length; i++) {
-			this.serializerFactories[i] = this.serializers[i].isStateful() ?
-					new RuntimeStatefulSerializerFactory<Object>(this.serializers[i], Object.class) :
-					new RuntimeStatelessSerializerFactory<Object>(this.serializers[i], Object.class);
-		}
-		
-		// set up auxiliary fields for normalized key support
-		this.normalizedKeyLengths = new int[keyPositions.length];
-		int nKeys = 0;
-		int nKeyLen = 0;
-		boolean inverted = false;
-		
-		for (int i = 0; i < this.keyPositions.length; i++) {
-			TypeComparator<?> k = this.comparators[i];
-			
-			// as long as the leading keys support normalized keys, we can build up the composite key
-			if (k.supportsNormalizedKey()) {
-				if (i == 0) {
-					// the first comparator decides whether we need to invert the key direction
-					inverted = k.invertNormalizedKey();
-				}
-				else if (k.invertNormalizedKey() != inverted) {
-					// if a successor does not agree on the inversion direction, it cannot be part of the normalized key
-					break;
-				}
-				
-				nKeys++;
-				final int len = k.getNormalizeKeyLen();
-				if (len < 0) {
-					throw new RuntimeException("Comparator " + k.getClass().getName() + " specifies an invalid length for the normalized key: " + len);
-				}
-				this.normalizedKeyLengths[i] = len;
-				nKeyLen += len;
-				
-				if (nKeyLen < 0) {
-					// overflow, which means we are out of budget for normalized key space anyways
-					nKeyLen = Integer.MAX_VALUE;
-					break;
-				}
-			} else {
-				break;
-			}
-		}
-		this.numLeadingNormalizableKeys = nKeys;
-		this.normalizableKeyPrefixLen = nKeyLen;
-		this.invertNormKey = inverted;
+		super(keyPositions, comparators, serializers);
+		extractedKeys = new Object[keyPositions.length];
 	}
 	
 	@SuppressWarnings("unchecked")
 	private TupleComparator(TupleComparator<T> toClone) {
-		// copy fields and serializer factories
-		this.keyPositions = toClone.keyPositions;
-		this.serializerFactories = toClone.serializerFactories;
-		
-		this.comparators = new TypeComparator[toClone.comparators.length];
-		for (int i = 0; i < toClone.comparators.length; i++) {
-			this.comparators[i] = toClone.comparators[i].duplicate();
-		}
-		
-		this.normalizedKeyLengths = toClone.normalizedKeyLengths;
-		this.numLeadingNormalizableKeys = toClone.numLeadingNormalizableKeys;
-		this.normalizableKeyPrefixLen = toClone.normalizableKeyPrefixLen;
-		this.invertNormKey = toClone.invertNormKey;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Comparator Methods
-	// --------------------------------------------------------------------------------------------
-	
-	protected int[] getKeyPositions() {
-		return this.keyPositions;
-	}
-	
-	protected TypeComparator<Object>[] getComparators() {
-		return this.comparators;
+		super(toClone);
+		extractedKeys = new Object[keyPositions.length];
+
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -207,28 +105,6 @@ public final class TupleComparator<T extends Tuple> extends TypeComparator<T> im
 			throw new KeyFieldOutOfBoundsException(keyPositions[i]);
 		}
 	}
-
-	@Override
-	public int compareToReference(TypeComparator<T> referencedComparator) {
-		TupleComparator<T> other = (TupleComparator<T>) referencedComparator;
-		
-		int i = 0;
-		try {
-			for (; i < this.keyPositions.length; i++) {
-				int cmp = this.comparators[i].compareToReference(other.comparators[i]);
-				if (cmp != 0) {
-					return cmp;
-				}
-			}
-			return 0;
-		}
-		catch (NullPointerException npex) {
-			throw new NullKeyFieldException(keyPositions[i]);
-		}
-		catch (IndexOutOfBoundsException iobex) {
-			throw new KeyFieldOutOfBoundsException(keyPositions[i]);
-		}
-	}
 	
 	@Override
 	public int compare(T first, T second) {
@@ -253,131 +129,32 @@ public final class TupleComparator<T extends Tuple> extends TypeComparator<T> im
 	}
 
 	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
-		if (deserializedFields1 == null) {
-			instantiateDeserializationUtils();
-		}
-		
-		int i = 0;
-		try {
-			for (; i < serializers.length; i++) {
-				deserializedFields1[i] = serializers[i].deserialize(deserializedFields1[i], firstSource);
-				deserializedFields2[i] = serializers[i].deserialize(deserializedFields2[i], secondSource);
-			}
-			
-			for (i = 0; i < keyPositions.length; i++) {
-				int keyPos = keyPositions[i];
-				int cmp = comparators[i].compare(deserializedFields1[keyPos], deserializedFields2[keyPos]);
-				if (cmp != 0) {
-					return cmp;
-				}
-			}
-			
-			return 0;
-		} catch (NullPointerException npex) {
-			throw new NullKeyFieldException(keyPositions[i]);
-		} catch (IndexOutOfBoundsException iobex) {
-			throw new KeyFieldOutOfBoundsException(keyPositions[i]);
-		}
-	}
-	
-	@Override
-	public boolean supportsNormalizedKey() {
-		return this.numLeadingNormalizableKeys > 0;
-	}
-
-	@Override
-	public int getNormalizeKeyLen() {
-		return this.normalizableKeyPrefixLen;
-	}
-
-	@Override
-	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-		return this.numLeadingNormalizableKeys < this.keyPositions.length ||
-				this.normalizableKeyPrefixLen == Integer.MAX_VALUE ||
-				this.normalizableKeyPrefixLen > keyBytes;
-	}
-
-	@Override
 	public void putNormalizedKey(T value, MemorySegment target, int offset, int numBytes) {
 		int i = 0;
 		try {
-			for (; i < this.numLeadingNormalizableKeys && numBytes > 0; i++)
-			{
-				int len = this.normalizedKeyLengths[i]; 
+			for (; i < this.numLeadingNormalizableKeys && numBytes > 0; i++) {
+				int len = this.normalizedKeyLengths[i];
 				len = numBytes >= len ? len : numBytes;
 				this.comparators[i].putNormalizedKey(value.getFieldNotNull(this.keyPositions[i]), target, offset, len);
 				numBytes -= len;
 				offset += len;
 			}
-		}
-		catch (NullFieldException nfex) {
+		} catch (NullFieldException nfex) {
 			throw new NullKeyFieldException(nfex);
-		}
-		catch (NullPointerException npex) {
+		} catch (NullPointerException npex) {
 			throw new NullKeyFieldException(this.keyPositions[i]);
 		}
 	}
 
 	@Override
-	public boolean invertNormalizedKey() {
-		return this.invertNormKey;
-	}
-	
-	
-	@Override
-	public boolean supportsSerializationWithKeyNormalization() {
-		return false;
-	}
-	
-	@Override
-	public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
-		throw new UnsupportedOperationException();
+	public Object[] extractKeys(T record) {
+		for (int i = 0; i < keyPositions.length; i++) {
+			extractedKeys[i] = record.getField(keyPositions[i]);
+		}
+		return extractedKeys;
 	}
 
-	@Override
-	public TupleComparator<T> duplicate() {
+	public TypeComparator<T> duplicate() {
 		return new TupleComparator<T>(this);
 	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@SuppressWarnings("unchecked")
-	private final void instantiateDeserializationUtils() {
-		if (this.serializers == null) {
-			this.serializers = new TypeSerializer[this.serializerFactories.length];
-			for (int i = 0; i < this.serializers.length; i++) {
-				this.serializers[i] = this.serializerFactories[i].getSerializer();
-			}
-		}
-		
-		this.deserializedFields1 = new Object[this.serializers.length];
-		this.deserializedFields2 = new Object[this.serializers.length];
-		
-		for (int i = 0; i < this.serializers.length; i++) {
-			this.deserializedFields1[i] = this.serializers[i].createInstance();
-			this.deserializedFields2[i] = this.serializers[i].createInstance();
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * A sequence of prime numbers to be used for salting the computed hash values.
-	 * Based on some empirical evidence, we are using a 32-element subsequence of the  
-	 * OEIS sequence #A068652 (numbers such that every cyclic permutation is a prime).
-	 * 
-	 * @see: http://en.wikipedia.org/wiki/List_of_prime_numbers
-	 * @see: http://oeis.org/A068652
-	 */
-	private static final int[] HASH_SALT = new int[] { 
-		73   , 79   , 97   , 113  , 131  , 197  , 199  , 311   , 
-		337  , 373  , 719  , 733  , 919  , 971  , 991  , 1193  , 
-		1931 , 3119 , 3779 , 7793 , 7937 , 9311 , 9377 , 11939 , 
-		19391, 19937, 37199, 39119, 71993, 91193, 93719, 93911 };
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
new file mode 100644
index 0000000..cea9879
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java
@@ -0,0 +1,281 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.types.KeyFieldOutOfBoundsException;
+import org.apache.flink.types.NullKeyFieldException;
+
+import java.io.IOException;
+
+
+public abstract class TupleComparatorBase<T> extends TypeComparator<T> implements java.io.Serializable {
+
+	/** key positions describe which fields are keys in what order */
+	protected int[] keyPositions;
+
+	/** comparators for the key fields, in the same order as the key fields */
+	protected TypeComparator[] comparators;
+
+	/** serializer factories to duplicate non thread-safe serializers */
+	protected TypeSerializerFactory<Object>[] serializerFactories;
+
+
+	protected int[] normalizedKeyLengths;
+
+	protected int numLeadingNormalizableKeys;
+
+	protected int normalizableKeyPrefixLen;
+
+	protected boolean invertNormKey;
+
+
+	/** serializers to deserialize the first n fields for comparison */
+	protected transient TypeSerializer[] serializers;
+
+	// cache for the deserialized field objects
+	protected transient Object[] deserializedFields1;
+	protected transient Object[] deserializedFields2;
+
+
+	@SuppressWarnings("unchecked")
+	public TupleComparatorBase(int[] keyPositions, TypeComparator<?>[] comparators, TypeSerializer<?>[] serializers) {
+		// set the default utils
+		this.keyPositions = keyPositions;
+		this.comparators = (TypeComparator<Object>[]) comparators;
+		this.serializers = (TypeSerializer<Object>[]) serializers;
+
+		// set the serializer factories.
+		this.serializerFactories = new TypeSerializerFactory[this.serializers.length];
+		for (int i = 0; i < serializers.length; i++) {
+			this.serializerFactories[i] = this.serializers[i].isStateful() ?
+					new RuntimeStatefulSerializerFactory<Object>(this.serializers[i], Object.class) :
+					new RuntimeStatelessSerializerFactory<Object>(this.serializers[i], Object.class);
+		}
+
+		// set up auxiliary fields for normalized key support
+		this.normalizedKeyLengths = new int[keyPositions.length];
+		int nKeys = 0;
+		int nKeyLen = 0;
+		boolean inverted = false;
+
+		for (int i = 0; i < this.keyPositions.length; i++) {
+			TypeComparator<?> k = this.comparators[i];
+
+			// as long as the leading keys support normalized keys, we can build up the composite key
+			if (k.supportsNormalizedKey()) {
+				if (i == 0) {
+					// the first comparator decides whether we need to invert the key direction
+					inverted = k.invertNormalizedKey();
+				}
+				else if (k.invertNormalizedKey() != inverted) {
+					// if a successor does not agree on the inversion direction, it cannot be part of the normalized key
+					break;
+				}
+
+				nKeys++;
+				final int len = k.getNormalizeKeyLen();
+				if (len < 0) {
+					throw new RuntimeException("Comparator " + k.getClass().getName() + " specifies an invalid length for the normalized key: " + len);
+				}
+				this.normalizedKeyLengths[i] = len;
+				nKeyLen += len;
+
+				if (nKeyLen < 0) {
+					// overflow, which means we are out of budget for normalized key space anyways
+					nKeyLen = Integer.MAX_VALUE;
+					break;
+				}
+			} else {
+				break;
+			}
+		}
+		this.numLeadingNormalizableKeys = nKeys;
+		this.normalizableKeyPrefixLen = nKeyLen;
+		this.invertNormKey = inverted;
+	}
+
+	@SuppressWarnings("unchecked")
+	protected TupleComparatorBase(TupleComparatorBase<T> toClone) {
+		privateDuplicate(toClone);
+	}
+
+	// We need this because we cannot call the cloning constructor from the
+	// ScalaTupleComparator
+	protected void privateDuplicate(TupleComparatorBase<T> toClone) {
+		// copy fields and serializer factories
+		this.keyPositions = toClone.keyPositions;
+		this.serializerFactories = toClone.serializerFactories;
+
+		this.comparators = new TypeComparator[toClone.comparators.length];
+		for (int i = 0; i < toClone.comparators.length; i++) {
+			this.comparators[i] = toClone.comparators[i].duplicate();
+		}
+
+		this.normalizedKeyLengths = toClone.normalizedKeyLengths;
+		this.numLeadingNormalizableKeys = toClone.numLeadingNormalizableKeys;
+		this.normalizableKeyPrefixLen = toClone.normalizableKeyPrefixLen;
+		this.invertNormKey = toClone.invertNormKey;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Comparator Methods
+	// --------------------------------------------------------------------------------------------
+	
+	protected int[] getKeyPositions() {
+		return this.keyPositions;
+	}
+	
+	public TypeComparator[] getComparators() {
+		return this.comparators;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Comparator Methods
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int compareToReference(TypeComparator<T> referencedComparator) {
+		TupleComparatorBase<T> other = (TupleComparatorBase<T>) referencedComparator;
+		
+		int i = 0;
+		try {
+			for (; i < this.keyPositions.length; i++) {
+				int cmp = this.comparators[i].compareToReference(other.comparators[i]);
+				if (cmp != 0) {
+					return cmp;
+				}
+			}
+			return 0;
+		}
+		catch (NullPointerException npex) {
+			throw new NullKeyFieldException(keyPositions[i]);
+		}
+		catch (IndexOutOfBoundsException iobex) {
+			throw new KeyFieldOutOfBoundsException(keyPositions[i]);
+		}
+	}
+	
+	@Override
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+		if (deserializedFields1 == null) {
+			instantiateDeserializationUtils();
+		}
+		
+		int i = 0;
+		try {
+			for (; i < serializers.length; i++) {
+				deserializedFields1[i] = serializers[i].deserialize(deserializedFields1[i], firstSource);
+				deserializedFields2[i] = serializers[i].deserialize(deserializedFields2[i], secondSource);
+			}
+			
+			for (i = 0; i < keyPositions.length; i++) {
+				int keyPos = keyPositions[i];
+				int cmp = comparators[i].compare(deserializedFields1[keyPos], deserializedFields2[keyPos]);
+				if (cmp != 0) {
+					return cmp;
+				}
+			}
+			
+			return 0;
+		} catch (NullPointerException npex) {
+			throw new NullKeyFieldException(keyPositions[i]);
+		} catch (IndexOutOfBoundsException iobex) {
+			throw new KeyFieldOutOfBoundsException(keyPositions[i]);
+		}
+	}
+	
+	@Override
+	public boolean supportsNormalizedKey() {
+		return this.numLeadingNormalizableKeys > 0;
+	}
+
+	@Override
+	public int getNormalizeKeyLen() {
+		return this.normalizableKeyPrefixLen;
+	}
+
+	@Override
+	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+		return this.numLeadingNormalizableKeys < this.keyPositions.length ||
+				this.normalizableKeyPrefixLen == Integer.MAX_VALUE ||
+				this.normalizableKeyPrefixLen > keyBytes;
+	}
+
+	@Override
+	public boolean invertNormalizedKey() {
+		return this.invertNormKey;
+	}
+	
+	
+	@Override
+	public boolean supportsSerializationWithKeyNormalization() {
+		return false;
+	}
+	
+	@Override
+	public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@SuppressWarnings("unchecked")
+	private final void instantiateDeserializationUtils() {
+		if (this.serializers == null) {
+			this.serializers = new TypeSerializer[this.serializerFactories.length];
+			for (int i = 0; i < this.serializers.length; i++) {
+				this.serializers[i] = this.serializerFactories[i].getSerializer();
+			}
+		}
+		
+		this.deserializedFields1 = new Object[this.serializers.length];
+		this.deserializedFields2 = new Object[this.serializers.length];
+		
+		for (int i = 0; i < this.serializers.length; i++) {
+			this.deserializedFields1[i] = this.serializers[i].createInstance();
+			this.deserializedFields2[i] = this.serializers[i].createInstance();
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * A sequence of prime numbers to be used for salting the computed hash values.
+	 * Based on some empirical evidence, we are using a 32-element subsequence of the  
+	 * OEIS sequence #A068652 (numbers such that every cyclic permutation is a prime).
+	 * 
+	 * @see: http://en.wikipedia.org/wiki/List_of_prime_numbers
+	 * @see: http://oeis.org/A068652
+	 */
+	protected static final int[] HASH_SALT = new int[] {
+		73   , 79   , 97   , 113  , 131  , 197  , 199  , 311   , 
+		337  , 373  , 719  , 733  , 919  , 971  , 991  , 1193  , 
+		1931 , 3119 , 3779 , 7793 , 7937 , 9311 , 9377 , 11939 , 
+		19391, 19937, 37199, 39119, 71993, 91193, 93719, 93911 };
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparator.java
deleted file mode 100644
index d63fccb..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldComparator.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.types.NullFieldException;
-import org.apache.flink.types.NullKeyFieldException;
-
-
-public final class TupleLeadingFieldComparator<T extends Tuple, K> extends TypeComparator<T>
-	implements java.io.Serializable
-{
-	private static final long serialVersionUID = 1L;
-	
-	private final TypeComparator<K> comparator;
-	
-	
-		
-	public TupleLeadingFieldComparator(TypeComparator<K> comparator) {
-		this.comparator = comparator;
-	}
-	
-	public TypeComparator<K> getComparator() {
-		return this.comparator;
-	}
-	
-	@Override
-	public int hash(T value) {
-		try {
-			return comparator.hash(value.<K> getFieldNotNull(0));
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-
-	}
-
-	@Override
-	public void setReference(T toCompare) {
-		try {
-			this.comparator.setReference(toCompare.<K> getFieldNotNull(0));
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-	
-
-	@Override
-	public boolean equalToReference(T candidate) {
-		try {
-			return this.comparator.equalToReference(candidate
-					.<K> getFieldNotNull(0));
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public int compareToReference(TypeComparator<T> referencedComparator) {
-		return this.comparator.compareToReference(((TupleLeadingFieldComparator<T, K>) referencedComparator).comparator);
-	}
-	
-	@Override
-	public int compare(T first, T second) {
-		try {
-			return this.comparator.compare(first.<K> getFieldNotNull(0),
-					second.<K> getFieldNotNull(0));
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-
-	@Override
-	public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
-		return this.comparator.compare(firstSource, secondSource);
-	}
-
-	@Override
-	public boolean supportsNormalizedKey() {
-		return this.comparator.supportsNormalizedKey();
-	}
-
-	@Override
-	public boolean supportsSerializationWithKeyNormalization() {
-		return false;
-	}
-
-	@Override
-	public int getNormalizeKeyLen() {
-		return this.comparator.getNormalizeKeyLen();
-	}
-
-	@Override
-	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-		return this.comparator.isNormalizedKeyPrefixOnly(keyBytes);
-	}
-
-	@Override
-	public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
-		try {
-			this.comparator.putNormalizedKey(record.<K> getFieldNotNull(0),
-					target, offset, numBytes);
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-
-	@Override
-	public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public boolean invertNormalizedKey() {
-		return this.comparator.invertNormalizedKey();
-	}
-
-	@Override
-	public TypeComparator<T> duplicate() {
-		return new TupleLeadingFieldComparator<T, K>(comparator.duplicate());
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	protected TypeComparator<K> getFieldComparator() {
-		return this.comparator;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparator.java
deleted file mode 100644
index 3611f70..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleLeadingFieldPairComparator.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.types.NullFieldException;
-import org.apache.flink.types.NullKeyFieldException;
-
-
-public class TupleLeadingFieldPairComparator<K, T1 extends Tuple, T2 extends Tuple> extends TypePairComparator<T1, T2> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	
-	private final TypeComparator<K> comparator1;
-	private final TypeComparator<K> comparator2;
-	
-	public TupleLeadingFieldPairComparator(TypeComparator<K> comparator1, TypeComparator<K> comparator2) {
-		this.comparator1 = comparator1;
-		this.comparator2 = comparator2;
-	}
-	
-	@Override
-	public void setReference(T1 reference) {
-		try {
-			this.comparator1.setReference(reference.<K> getFieldNotNull(0));
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-
-	@Override
-	public boolean equalToReference(T2 candidate) {
-		try {
-			return this.comparator1.equalToReference(candidate
-					.<K> getFieldNotNull(0));
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-
-	@Override
-	public int compareToReference(T2 candidate) {
-		try {
-			this.comparator2.setReference(candidate.<K> getFieldNotNull(0));
-			return this.comparator1.compareToReference(this.comparator2);
-		} catch (NullFieldException nfex) {
-			throw new NullKeyFieldException(nfex);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/57b8e66a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparator.java
deleted file mode 100644
index 4c4094d..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TuplePairComparator.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.types.NullFieldException;
-import org.apache.flink.types.NullKeyFieldException;
-
-
-public class TuplePairComparator<T1 extends Tuple, T2 extends Tuple> extends TypePairComparator<T1, T2> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	
-	private final int[] keyFields1, keyFields2;
-	private final TypeComparator<Object>[] comparators1;
-	private final TypeComparator<Object>[] comparators2;
-	
-	@SuppressWarnings("unchecked")
-	public TuplePairComparator(int[] keyFields1, int[] keyFields2, TypeComparator<Object>[] comparators1, TypeComparator<Object>[] comparators2) {
-		
-		if(keyFields1.length != keyFields2.length 
-			|| keyFields1.length != comparators1.length
-			|| keyFields2.length != comparators2.length) {
-			
-			throw new IllegalArgumentException("Number of key fields and comparators differ.");
-		}
-		
-		int numKeys = keyFields1.length;
-		
-		this.keyFields1 = keyFields1;
-		this.keyFields2 = keyFields2;
-		this.comparators1 = new TypeComparator[numKeys];
-		this.comparators2 = new TypeComparator[numKeys];
-		
-		for(int i = 0; i < numKeys; i++) {
-			this.comparators1[i] = comparators1[i].duplicate();
-			this.comparators2[i] = comparators2[i].duplicate();
-		}
-	}
-	
-	@Override
-	public void setReference(T1 reference) {
-		for (int i = 0; i < this.comparators1.length; i++) {
-			try {
-				this.comparators1[i].setReference(reference
-						.getFieldNotNull(keyFields1[i]));
-			} catch (NullFieldException nfex) {
-				throw new NullKeyFieldException(nfex);
-			}
-		}
-	}
-
-	@Override
-	public boolean equalToReference(T2 candidate) {
-		for (int i = 0; i < this.comparators1.length; i++) {
-			try {
-				if (!this.comparators1[i].equalToReference(candidate
-						.getFieldNotNull(keyFields2[i]))) {
-					return false;
-				}
-			} catch (NullFieldException nfex) {
-				throw new NullKeyFieldException(nfex);
-			}
-		}
-		return true;
-	}
-
-	@Override
-	public int compareToReference(T2 candidate) {
-		for (int i = 0; i < this.comparators1.length; i++) {
-			try {
-				this.comparators2[i].setReference(candidate
-						.getFieldNotNull(keyFields2[i]));
-				int res = this.comparators1[i]
-						.compareToReference(this.comparators2[i]);
-				if (res != 0) {
-					return res;
-				}
-			} catch (NullFieldException nfex) {
-				throw new NullKeyFieldException(nfex);
-			}
-		}
-		return 0;
-	}
-}


[25/60] Renamed java examples package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
deleted file mode 100644
index 3bd6c18..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogData.java
+++ /dev/null
@@ -1,428 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.relational.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * Provides the default data sets used for the Weblog Analysis example program.
- * The default data sets are used, if no parameters are given to the program.
- *
- */
-public class WebLogData {
-
-	public static final Object [][] DOCUMENTS = {
-			new Object [] {"url_0","dolor ad amet enim laoreet nostrud veniam aliquip ex nonummy diam dolore tincidunt tation exerci exerci wisi dolor nostrud "},
-			new Object [] {"url_1","wisi minim adipiscing nibh adipiscing ut nibh Lorem Ut nonummy euismod nibh wisi sit consectetuer exerci sed aliquip aliquip dolore aliquam enim dolore veniam aliquam euismod suscipit ad adipiscing exerci aliquip consectetuer euismod aliquip ad exerci ex nibh ex erat exerci laoreet lobortis quis "},
-			new Object [] {"url_2","diam sed convection aliquip amet commodo nonummy sed sed commodo commodo diam commodo adipiscing ad exerci magna exerci tation quis lobortis "},
-			new Object [] {"url_3","exerci suscipit sed lobortis amet lobortis aliquip nibh nostrud ad convection commodo ad nibh sed minim amet ad ea ea "},
-			new Object [] {"url_4","sit enim dolor quis laoreet ullamcorper veniam adipiscing ex quis commodo "},
-			new Object [] {"url_5","elit aliquip ea nisl oscillations sit dolor ipsum tincidunt ullamcorper dolore enim adipiscing laoreet elit ea volutpat adipiscing ea nibh nostrud Ut aliquam veniam Lorem laoreet veniam aliquip "},
-			new Object [] {"url_6","consectetuer ad sed suscipit euismod aliquip quis ullamcorper oscillations tation consectetuer tation amet suscipit nibh enim nonummy veniam commodo commodo diam euismod dolor Ut aliquip diam ex ad nonummy ad tincidunt minim exerci consectetuer veniam convection aliquam ut ut Lorem euismod sed ipsum volutpat "},
-			new Object [] {"url_7","Ut volutpat veniam ut consectetuer diam ut aliquam dolor nostrud erat consectetuer adipiscing exerci consectetuer Ut ullamcorper suscipit aliquam sed dolor nisl "},
-			new Object [] {"url_8","suscipit amet wisi nisl veniam lobortis sit Lorem aliquam nostrud aliquam ipsum ut laoreet suscipit Lorem laoreet editors adipiscing ullamcorper veniam erat consectetuer ut lobortis dolore elit sed tincidunt ipsum tation ullamcorper nonummy adipiscing ex ad laoreet ipsum suscipit lobortis lobortis Ut nonummy adipiscing erat volutpat aliquam "},
-			new Object [] {"url_9","nonummy commodo tation editors ut quis sit quis lobortis ea dolore oscillations diam ad dolor lobortis nisl ad veniam ullamcorper quis magna volutpat sit ipsum consectetuer dolore exerci commodo magna erat enim ut suscipit "},
-			new Object [] {"url_10","amet erat magna consectetuer tation tation aliquip nibh aliquam sed adipiscing ut commodo ex erat tincidunt aliquam ipsum Ut Ut sit tincidunt adipiscing suscipit minim sed erat dolor consectetuer Lorem consectetuer Lorem amet nibh diam ea ex enim suscipit wisi dolor nonummy magna enim euismod ullamcorper ut suscipit adipiscing "},
-			new Object [] {"url_11","ex quis exerci tation diam elit nostrud nostrud ut ipsum elit amet diam laoreet amet consectetuer volutpat sed lobortis "},
-			new Object [] {"url_12","elit suscipit sit ullamcorper ut ad erat ut dolor nostrud quis nisl enim erat dolor convection ad minim ut veniam nostrud sed editors adipiscing volutpat Ut aliquip commodo sed euismod adipiscing erat adipiscing dolore nostrud minim sed lobortis ea diam "},
-			new Object [] {"url_13","enim ut quis commodo veniam minim erat lobortis ad diam ex dolor tincidunt exerci ut aliquip tincidunt minim ut magna sed enim wisi veniam oscillations Lorem consectetuer "},
-			new Object [] {"url_14","nibh ipsum ullamcorper volutpat ut wisi dolor quis amet euismod quis ipsum ipsum minim tation volutpat sit exerci volutpat amet nonummy euismod veniam consectetuer sit consectetuer tincidunt nibh aliquam lobortis tation veniam ut ullamcorper wisi magna Ut volutpat consectetuer erat quis dolore ea tation "},
-			new Object [] {"url_15","ad wisi sed enim aliquam oscillations nibh Lorem lobortis veniam nibh laoreet nonummy sed nibh Lorem adipiscing diam magna nostrud magna oscillations ut oscillations elit nostrud diam editors Lorem "},
-			new Object [] {"url_16","nostrud volutpat veniam exerci tincidunt nostrud quis elit ipsum ea nonummy volutpat dolor elit lobortis magna nisl ut ullamcorper magna Lorem exerci nibh nisl magna editors erat aliquam aliquam ullamcorper sit aliquam sit nostrud oscillations consectetuer adipiscing suscipit convection exerci ea ullamcorper ex nisl "},
-			new Object [] {"url_17","ad ex aliquam erat aliquam elit veniam laoreet ut amet amet nostrud ut adipiscing Ut Lorem suscipit ex magna ullamcorper aliquam ullamcorper ullamcorper amet amet commodo aliquam volutpat nonummy nonummy tincidunt amet tation tincidunt volutpat ut veniam nisl erat dolor enim nonummy nostrud adipiscing laoreet adipiscing "},
-			new Object [] {"url_18","lobortis ipsum ex tincidunt tincidunt editors euismod consectetuer ipsum adipiscing lobortis exerci adipiscing nonummy nisl dolore nonummy erat exerci nisl ut dolore wisi volutpat lobortis magna "},
-			new Object [] {"url_19","ipsum tation laoreet tation adipiscing wisi nibh diam Ut suscipit ad wisi "},
-			new Object [] {"url_20","diam Lorem enim wisi ad lobortis dolor Ut ipsum amet dolore consectetuer nisl exerci nisl nonummy minim Ut erat oscillations ut Lorem nostrud dolore Ut dolore exerci ad ipsum dolore ex dolore aliquip sed aliquam ex aliquip magna amet ex dolore oscillations aliquip tation magna Ut "},
-			new Object [] {"url_21","lobortis ut amet ex nisl ullamcorper tincidunt ut elit diam quis suscipit ad amet ipsum magna Ut ex tincidunt "},
-			new Object [] {"url_22","amet commodo nisl ad quis lobortis ut commodo sit ut erat exerci lobortis suscipit nibh ut nostrud ut adipiscing commodo commodo quis quis nostrud nisl ipsum nostrud laoreet Lorem nostrud erat nostrud amet consectetuer laoreet oscillations wisi sit magna nibh amet "},
-			new Object [] {"url_23","adipiscing suscipit suscipit aliquip suscipit consectetuer minim magna ea erat nibh sit suscipit sed dolor oscillations nonummy volutpat ut tincidunt "},
-			new Object [] {"url_24","commodo sed tincidunt aliquip aliquip dolore commodo nonummy sed erat ut ex exerci dolore adipiscing tincidunt ex diam amet aliquam "},
-			new Object [] {"url_25","consectetuer consectetuer exerci quis ea veniam aliquam laoreet minim ex "},
-			new Object [] {"url_26","dolor exerci euismod minim magna quis erat consectetuer sed ex erat dolore quis ut oscillations ullamcorper Lorem exerci ex nibh ut exerci ullamcorper veniam nibh ut commodo ut Ut nostrud tincidunt tincidunt ad dolore Lorem ea tation enim erat nibh ut ea nonummy sed sed wisi nisl dolore "},
-			new Object [] {"url_27","amet elit ea ea nostrud editors Ut nostrud amet laoreet adipiscing ut nisl nonummy tincidunt ea ipsum ex dolore dolore oscillations sit minim Ut wisi ut laoreet minim elit "},
-			new Object [] {"url_28","wisi exerci volutpat Ut nostrud euismod minim Ut sit euismod ut ea magna consectetuer nisl ad minim tation nisl adipiscing Lorem aliquam quis exerci erat minim aliquip sit Lorem wisi wisi ut "},
-			new Object [] {"url_29","amet sed laoreet amet aliquam minim enim tincidunt Lorem sit aliquip amet suscipit ut laoreet elit suscipit erat ut tincidunt suscipit ipsum sed euismod elit dolore euismod dolore ut dolor nostrud ipsum tincidunt commodo adipiscing aliquam ut wisi dolor dolor suscipit "},
-			new Object [] {"url_30","euismod Lorem ex tincidunt amet enim minim suscipit exerci diam veniam amet nostrud ea ea "},
-			new Object [] {"url_31","ex ipsum sit euismod euismod ullamcorper tincidunt ut wisi ea adipiscing sed diam tation ipsum dolor aliquam veniam nonummy aliquip aliquip Lorem ut minim nisl tation sit exerci ullamcorper Ut dolor euismod aliquam consectetuer ad nonummy commodo exerci "},
-			new Object [] {"url_32","volutpat ipsum lobortis nisl veniam minim adipiscing dolor editors quis nostrud amet nostrud "},
-			new Object [] {"url_33","commodo wisi aliquip ut aliquam sed nostrud ex diam ad nostrud enim ut amet enim ea ad sed tation nostrud suscipit ea magna magna Lorem amet lobortis ut quis nibh aliquam aliquam exerci aliquip lobortis consectetuer enim wisi ea nisl laoreet erat dolore "},
-			new Object [] {"url_34","tincidunt adipiscing enim tation nibh Ut dolore tincidunt tation laoreet suscipit minim aliquam volutpat laoreet suscipit tincidunt nibh ut ut sit nostrud nonummy tincidunt exerci sit ad sed consectetuer minim dolor dolore laoreet nostrud nibh laoreet ea adipiscing exerci dolore ipsum "},
-			new Object [] {"url_35","tation ut erat ut tation dolor Lorem laoreet Lorem elit adipiscing wisi aliquip nostrud elit Ut volutpat ea aliquam aliquip "},
-			new Object [] {"url_36","lobortis enim ullamcorper adipiscing consectetuer aliquip wisi enim minim Ut minim elit elit aliquam exerci ullamcorper amet lobortis adipiscing diam laoreet consectetuer nostrud diam diam amet ut enim ullamcorper aliquip diam ut nostrud diam magna amet nonummy commodo wisi enim ullamcorper suscipit euismod dolore tincidunt magna suscipit elit "},
-			new Object [] {"url_37","elit adipiscing nisl nisl ex aliquip nibh sed ut ad Lorem elit consectetuer ad volutpat lobortis amet veniam ipsum nibh ut consectetuer editors ad aliquam "},
-			new Object [] {"url_38","elit quis nibh adipiscing sit consectetuer ut euismod quis tincidunt quis nisl consectetuer dolor diam suscipit quis dolore Lorem suscipit nonummy sed ex "},
-			new Object [] {"url_39","nisl sit consectetuer elit oscillations enim ipsum enim nostrud adipiscing nostrud editors aliquam "},
-			new Object [] {"url_40","sed wisi dolor diam commodo ullamcorper commodo nostrud ullamcorper laoreet minim dolore suscipit laoreet tation aliquip "},
-			new Object [] {"url_41","ad consectetuer exerci nisl exerci amet enim diam lobortis Lorem ex volutpat volutpat nibh aliquam ut ullamcorper volutpat nostrud ut adipiscing ullamcorper "},
-			new Object [] {"url_42","minim laoreet tation magna veniam ut ea sit ipsum tincidunt Ut amet ex aliquip ex euismod exerci wisi elit editors ad amet veniam ad editors "},
-			new Object [] {"url_43","ut nisl ad ullamcorper nibh Ut editors exerci enim exerci ea laoreet veniam ea amet exerci volutpat amet ad "},
-			new Object [] {"url_44","volutpat tincidunt enim amet sed tincidunt consectetuer ullamcorper nisl Ut adipiscing tation ad ad amet nonummy elit erat nibh Lorem erat elit laoreet consectetuer sed aliquip nostrud "},
-			new Object [] {"url_45","sed aliquam ut ut consectetuer wisi euismod enim erat euismod quis exerci amet tation sit "},
-			new Object [] {"url_46","lobortis oscillations tation aliquam dolore Lorem aliquip tation exerci ullamcorper aliquam aliquip lobortis ex tation dolor ut ut sed suscipit nisl ullamcorper sed editors laoreet aliquip enim dolor veniam tincidunt sed euismod tation "},
-			new Object [] {"url_47","Lorem Lorem ut wisi ad ut tation consectetuer exerci convection tation ullamcorper sed dolore quis aliquam ipsum lobortis commodo nonummy "},
-			new Object [] {"url_48","laoreet minim veniam nisl elit sit amet commodo ex ullamcorper suscipit aliquip laoreet convection Ut ex minim aliquam "},
-			new Object [] {"url_49","lobortis nonummy minim amet sit veniam quis consectetuer tincidunt laoreet quis "},
-			new Object [] {"url_50","lobortis nisl commodo dolor amet nibh editors enim magna minim elit euismod diam laoreet laoreet ad minim sed ut Ut lobortis adipiscing quis sed ut aliquam oscillations exerci tation consectetuer lobortis elit tincidunt consectetuer minim amet dolore quis aliquam Ut exerci sed aliquam quis quis ullamcorper Ut ex tincidunt "},
-			new Object [] {"url_51","nostrud nisl ea erat ut suscipit Ut sit oscillations ullamcorper nonummy magna lobortis dolore editors tincidunt nostrud suscipit ex quis tation ut sit amet nostrud laoreet ex tincidunt "},
-			new Object [] {"url_52","ea tation commodo elit sed ex sed quis enim nisl magna laoreet adipiscing amet sit nostrud consectetuer nibh tincidunt veniam ex veniam euismod exerci sed dolore suscipit nisl tincidunt euismod quis Ut enim euismod dolor diam exerci magna exerci ut exerci nisl "},
-			new Object [] {"url_53","volutpat amet Ut lobortis dolor tation minim nonummy lobortis convection nostrud "},
-			new Object [] {"url_54","ullamcorper commodo Ut amet sit nostrud aliquam ad amet wisi enim nostrud ipsum nisl veniam erat aliquam ex aliquam dolor dolor ut consectetuer euismod exerci elit exerci Ut ea minim enim consectetuer ad consectetuer nonummy convection adipiscing ad ullamcorper lobortis nonummy laoreet nonummy aliquam ullamcorper ad nostrud amet "},
-			new Object [] {"url_55","wisi magna editors amet aliquam diam amet aliquip nisl consectetuer laoreet nonummy suscipit euismod diam enim tation elit ut lobortis quis euismod suscipit nostrud ea ea commodo lobortis dolore Ut nisl nostrud dolor laoreet euismod ea dolore aliquam ut Lorem exerci ex sit "},
-			new Object [] {"url_56","ex dolor veniam wisi laoreet ut exerci diam ad ex ut ut laoreet ut nisl ullamcorper nisl "},
-			new Object [] {"url_57","diam adipiscing Ut ut Lorem amet erat elit erat magna adipiscing euismod elit ullamcorper nostrud aliquam dolor ullamcorper sit tation tation "},
-			new Object [] {"url_58","laoreet convection veniam lobortis dolore ut nonummy commodo erat lobortis veniam nostrud dolore minim commodo ut consectetuer magna erat ea dolore Lorem suscipit ex ipsum exerci sed enim ea tation suscipit enim adipiscing "},
-			new Object [] {"url_59","amet ut ut Ut ad dolor quis ad magna exerci suscipit magna nibh commodo euismod amet euismod wisi diam suscipit dolore Lorem dolor ex amet exerci aliquip ut ut lobortis quis elit minim sed Lorem "},
-			new Object [] {"url_60","ut ut amet ullamcorper amet euismod dolor amet elit exerci adipiscing sed suscipit sed exerci wisi diam veniam wisi suscipit ut quis nibh ullamcorper ex quis magna dolore volutpat editors minim ut sit aliquip oscillations nisl ipsum "},
-			new Object [] {"url_61","nibh nostrud tincidunt lobortis adipiscing adipiscing ullamcorper ullamcorper ipsum nisl ullamcorper aliquip laoreet commodo ut tation wisi diam commodo aliquip commodo suscipit tincidunt volutpat elit enim laoreet ut nostrud ad nonummy ipsum "},
-			new Object [] {"url_62","Ut ut minim enim amet euismod erat elit commodo consectetuer Ut quis dolor ex diam quis wisi tation tincidunt laoreet volutpat "},
-			new Object [] {"url_63","ut erat volutpat euismod amet ea nonummy lobortis ut Ut ea veniam sed veniam nostrud "},
-			new Object [] {"url_64","tation dolor suscipit minim nisl wisi consectetuer aliquip tation Ut commodo ut dolore consectetuer elit wisi nisl ipsum "},
-			new Object [] {"url_65","ullamcorper nisl Lorem magna tation veniam aliquam diam amet euismod "},
-			new Object [] {"url_66","euismod aliquam tincidunt Ut volutpat ea lobortis sit ut volutpat ut lobortis ut lobortis ut nisl amet dolor sed ipsum enim ullamcorper diam euismod nostrud wisi erat quis diam nibh Ut dolore sed amet tation enim diam "},
-			new Object [] {"url_67","amet minim minim amet laoreet Lorem aliquam veniam elit volutpat magna adipiscing enim enim euismod laoreet sed ex sed aliquam ad ea ut adipiscing suscipit ex minim dolore minim ea laoreet nisl "},
-			new Object [] {"url_68","aliquam ea volutpat ut wisi tation tation nibh nisl erat laoreet ea volutpat dolor dolor aliquam exerci quis ullamcorper aliquam ut quis suscipit "},
-			new Object [] {"url_69","quis exerci ut aliquip wisi dolore magna nibh consectetuer magna tation ullamcorper lobortis sed amet adipiscing minim suscipit nibh nibh nostrud euismod enim "},
-			new Object [] {"url_70","tation enim consectetuer adipiscing wisi laoreet diam aliquip nostrud elit nostrud aliquip ea minim amet diam dolore "},
-			new Object [] {"url_71","consectetuer tincidunt nibh amet tation nonummy sit tation diam sed diam tation "},
-			new Object [] {"url_72","Lorem ut nostrud nonummy minim quis euismod lobortis nostrud nonummy adipiscing tincidunt consectetuer ut nibh ad suscipit dolor ut elit dolore amet ut quis tation ullamcorper nonummy laoreet ullamcorper aliquam dolore convection dolor tincidunt ut ullamcorper ex dolor suscipit erat oscillations ad "},
-			new Object [] {"url_73","elit Ut commodo ut ullamcorper ullamcorper ut euismod commodo diam aliquip suscipit consectetuer exerci tation nostrud ut wisi exerci sed ut elit sed volutpat Lorem nibh laoreet consectetuer ex Lorem elit aliquam commodo lobortis ad "},
-			new Object [] {"url_74","quis magna laoreet commodo aliquam nisl ullamcorper veniam tation wisi consectetuer commodo consectetuer ad dolore aliquam dolor elit amet sit amet nibh commodo erat veniam aliquip dolore ad magna ad ipsum Ut exerci ea volutpat nisl amet nostrud sit "},
-			new Object [] {"url_75","tincidunt suscipit sit aliquip aliquam adipiscing dolore exerci Ut suscipit ut sit laoreet suscipit wisi sit enim nonummy consectetuer dolore editors "},
-			new Object [] {"url_76","veniam ullamcorper tation sit suscipit dolor suscipit veniam sit Lorem quis sed nostrud ad tincidunt elit adipiscing "},
-			new Object [] {"url_77","volutpat sit amet veniam quis ipsum nibh elit enim commodo magna veniam magna convection "},
-			new Object [] {"url_78","tation dolore minim elit nisl volutpat tation laoreet enim nostrud exerci dolore tincidunt aliquip Lorem ipsum nostrud quis adipiscing ullamcorper erat lobortis tation commodo Ut ipsum commodo magna ad ipsum ut enim "},
-			new Object [] {"url_79","lobortis amet elit Lorem amet nonummy commodo tation ex ea amet Lorem ea nonummy commodo veniam volutpat nibh wisi ad ipsum euismod ea convection nostrud nisl erat veniam Ut aliquip ad aliquip editors wisi magna tation nostrud nonummy adipiscing ullamcorper aliquip "},
-			new Object [] {"url_80","tincidunt nostrud nostrud magna ea euismod ea consectetuer nisl exerci ea dolor nisl commodo ex erat ipsum exerci suscipit ad nisl ea nonummy suscipit adipiscing laoreet sit euismod nibh adipiscing sed minim commodo amet "},
-			new Object [] {"url_81","nostrud erat ut sed editors erat amet magna lobortis diam laoreet dolor amet nibh ut ipsum ipsum amet ut sed ut exerci elit suscipit wisi magna ut veniam nisl commodo enim adipiscing laoreet ad Lorem oscillations "},
-			new Object [] {"url_82","quis commodo nibh nibh volutpat suscipit dolore magna tincidunt nibh ut ad ullamcorper ullamcorper quis enim ad ut tation minim laoreet veniam dolor sed tincidunt exerci exerci nostrud ullamcorper amet ut ut ullamcorper "},
-			new Object [] {"url_83","sit suscipit volutpat elit tation elit sed sed dolor ex ex ipsum euismod laoreet magna lobortis ad "},
-			new Object [] {"url_84","lobortis ipsum euismod enim ea tation veniam tation oscillations aliquip consectetuer euismod ut sed lobortis tation oscillations commodo euismod laoreet suscipit amet elit ullamcorper volutpat aliquam ea enim ullamcorper consectetuer laoreet tation quis ut commodo erat euismod dolor laoreet ullamcorper laoreet "},
-			new Object [] {"url_85","adipiscing sit quis commodo consectetuer quis enim euismod exerci nonummy ea nostrud Ut veniam sit aliquip nisl enim "},
-			new Object [] {"url_86","nostrud dolore veniam veniam wisi aliquip adipiscing diam sed quis ullamcorper "},
-			new Object [] {"url_87","quis Lorem suscipit Ut nibh diam euismod consectetuer lobortis ipsum sed suscipit consectetuer euismod laoreet ut wisi nisl elit quis commodo adipiscing adipiscing suscipit aliquam nisl quis magna ipsum enim ad quis ea magna Lorem nibh ea "},
-			new Object [] {"url_88","euismod commodo sed tincidunt Ut veniam consectetuer quis erat ex ea erat laoreet commodo nibh minim "},
-			new Object [] {"url_89","tation diam editors Ut enim nibh Lorem volutpat quis diam suscipit exerci wisi ad "},
-			new Object [] {"url_90","volutpat editors ea nibh wisi ad amet volutpat nisl ullamcorper nibh volutpat minim ex ut sit veniam Lorem consectetuer quis ad sit suscipit volutpat wisi diam sed tincidunt ipsum minim convection ea diam oscillations quis lobortis "},
-			new Object [] {"url_91","enim minim nonummy ea minim euismod adipiscing editors volutpat magna sit magna ut ipsum ut "},
-			new Object [] {"url_92","nisl Ut commodo amet euismod lobortis ea ea wisi commodo Lorem sit ipsum volutpat nonummy exerci erat elit exerci magna ad erat enim laoreet quis nostrud wisi ut veniam amet ullamcorper lobortis ad suscipit volutpat veniam nostrud nibh quis ipsum dolore consectetuer veniam ipsum aliquip dolore sed laoreet ipsum "},
-			new Object [] {"url_93","nonummy aliquam ad lobortis Lorem erat ad tation Lorem exerci ex "},
-			new Object [] {"url_94","nonummy dolore commodo exerci ex quis ut suscipit elit laoreet sit tation magna veniam ea sit nonummy veniam Lorem quis nibh aliquip exerci amet ullamcorper adipiscing erat nisl editors diam commodo ad euismod adipiscing ea suscipit exerci aliquip volutpat tation enim volutpat sit "},
-			new Object [] {"url_95","sit suscipit oscillations ipsum nibh dolor ea dolore ea elit ipsum minim editors magna consectetuer ullamcorper commodo nonummy sit nostrud aliquip sit erat ullamcorper ullamcorper nibh veniam erat quis dolore nonummy "},
-			new Object [] {"url_96","nostrud quis ut volutpat magna ad quis adipiscing Lorem commodo exerci laoreet magna adipiscing erat quis wisi ea ea laoreet enim convection ad dolor nisl amet nibh aliquam adipiscing tincidunt minim diam Lorem commodo adipiscing volutpat "},
-			new Object [] {"url_97","laoreet laoreet suscipit nostrud dolore adipiscing volutpat Ut sed nisl diam ullamcorper ex ut ut dolor amet nostrud euismod dolore veniam veniam enim tation veniam ea minim minim volutpat tincidunt "},
-			new Object [] {"url_98","quis lobortis amet wisi nostrud ipsum aliquam convection tincidunt dolore ullamcorper nibh lobortis volutpat ea nostrud oscillations minim nonummy enim ad lobortis exerci ipsum ullamcorper nibh nonummy diam amet enim veniam ut nostrud "},
-			new Object [] {"url_99","aliquam wisi suscipit commodo diam amet amet magna nisl enim nostrud tation nisl nostrud nibh ut "}
-	};
-
-	public static final Object [][] RANKS = {
-			new Object [] {30,"url_0",43},
-			new Object [] {82,"url_1",39},
-			new Object [] {56,"url_2",31},
-			new Object [] {96,"url_3",36},
-			new Object [] {31,"url_4",36},
-			new Object [] {29,"url_5",6},
-			new Object [] {33,"url_6",48},
-			new Object [] {66,"url_7",40},
-			new Object [] {28,"url_8",51},
-			new Object [] {9,"url_9",4},
-			new Object [] {49,"url_10",24},
-			new Object [] {26,"url_11",12},
-			new Object [] {39,"url_12",46},
-			new Object [] {84,"url_13",53},
-			new Object [] {29,"url_14",50},
-			new Object [] {21,"url_15",12},
-			new Object [] {69,"url_16",34},
-			new Object [] {11,"url_17",38},
-			new Object [] {96,"url_18",13},
-			new Object [] {56,"url_19",48},
-			new Object [] {18,"url_20",36},
-			new Object [] {31,"url_21",21},
-			new Object [] {29,"url_22",11},
-			new Object [] {71,"url_23",30},
-			new Object [] {85,"url_24",48},
-			new Object [] {19,"url_25",45},
-			new Object [] {69,"url_26",9},
-			new Object [] {20,"url_27",51},
-			new Object [] {33,"url_28",46},
-			new Object [] {75,"url_29",38},
-			new Object [] {96,"url_30",51},
-			new Object [] {73,"url_31",40},
-			new Object [] {67,"url_32",16},
-			new Object [] {24,"url_33",24},
-			new Object [] {27,"url_34",35},
-			new Object [] {33,"url_35",35},
-			new Object [] {7,"url_36",22},
-			new Object [] {83,"url_37",41},
-			new Object [] {23,"url_38",49},
-			new Object [] {41,"url_39",33},
-			new Object [] {66,"url_40",38},
-			new Object [] {4,"url_41",52},
-			new Object [] {34,"url_42",4},
-			new Object [] {28,"url_43",12},
-			new Object [] {14,"url_44",14},
-			new Object [] {41,"url_45",11},
-			new Object [] {48,"url_46",37},
-			new Object [] {75,"url_47",41},
-			new Object [] {78,"url_48",3},
-			new Object [] {63,"url_49",28}
-	};
-
-
-	public static final Object [][] VISITS = {
-			new Object [] {"url_2","2003-12-17"},
-			new Object [] {"url_9","2008-11-11"},
-			new Object [] {"url_14","2003-11-5"},
-			new Object [] {"url_46","2009-2-16"},
-			new Object [] {"url_14","2004-11-9"},
-			new Object [] {"url_36","2001-3-9"},
-			new Object [] {"url_35","2006-8-13"},
-			new Object [] {"url_22","2008-1-18"},
-			new Object [] {"url_36","2002-3-9"},
-			new Object [] {"url_13","2007-7-17"},
-			new Object [] {"url_23","2009-6-16"},
-			new Object [] {"url_16","2000-7-15"},
-			new Object [] {"url_41","2002-5-10"},
-			new Object [] {"url_6","2004-11-9"},
-			new Object [] {"url_5","2003-6-7"},
-			new Object [] {"url_22","2002-11-5"},
-			new Object [] {"url_11","2007-7-21"},
-			new Object [] {"url_38","2009-12-2"},
-			new Object [] {"url_6","2004-11-2"},
-			new Object [] {"url_46","2000-6-4"},
-			new Object [] {"url_34","2003-9-2"},
-			new Object [] {"url_31","2008-2-24"},
-			new Object [] {"url_0","2003-2-2"},
-			new Object [] {"url_47","2003-7-8"},
-			new Object [] {"url_49","2009-9-13"},
-			new Object [] {"url_11","2003-4-2"},
-			new Object [] {"url_20","2000-6-18"},
-			new Object [] {"url_38","2000-2-22"},
-			new Object [] {"url_44","2009-2-17"},
-			new Object [] {"url_26","2000-6-21"},
-			new Object [] {"url_13","2000-11-25"},
-			new Object [] {"url_47","2005-4-19"},
-			new Object [] {"url_46","2008-1-7"},
-			new Object [] {"url_33","2004-12-24"},
-			new Object [] {"url_32","2009-2-8"},
-			new Object [] {"url_26","2000-9-21"},
-			new Object [] {"url_9","2002-8-18"},
-			new Object [] {"url_38","2002-11-27"},
-			new Object [] {"url_37","2008-2-26"},
-			new Object [] {"url_1","2007-3-22"},
-			new Object [] {"url_37","2002-3-20"},
-			new Object [] {"url_27","2008-11-12"},
-			new Object [] {"url_30","2000-12-16"},
-			new Object [] {"url_48","2000-12-17"},
-			new Object [] {"url_46","2008-4-16"},
-			new Object [] {"url_29","2006-3-9"},
-			new Object [] {"url_0","2007-7-26"},
-			new Object [] {"url_46","2009-12-15"},
-			new Object [] {"url_34","2002-2-13"},
-			new Object [] {"url_24","2009-3-1"},
-			new Object [] {"url_43","2007-11-4"},
-			new Object [] {"url_3","2004-2-16"},
-			new Object [] {"url_26","2000-10-26"},
-			new Object [] {"url_42","2004-7-14"},
-			new Object [] {"url_13","2004-9-10"},
-			new Object [] {"url_21","2000-2-21"},
-			new Object [] {"url_9","2006-6-5"},
-			new Object [] {"url_46","2001-12-17"},
-			new Object [] {"url_24","2006-12-8"},
-			new Object [] {"url_25","2006-9-2"},
-			new Object [] {"url_37","2002-6-26"},
-			new Object [] {"url_18","2006-6-2"},
-			new Object [] {"url_46","2003-5-24"},
-			new Object [] {"url_32","2000-10-17"},
-			new Object [] {"url_45","2002-1-12"},
-			new Object [] {"url_12","2005-12-13"},
-			new Object [] {"url_49","2009-3-9"},
-			new Object [] {"url_31","2001-9-19"},
-			new Object [] {"url_22","2002-7-9"},
-			new Object [] {"url_27","2005-2-3"},
-			new Object [] {"url_43","2008-7-15"},
-			new Object [] {"url_20","2000-3-23"},
-			new Object [] {"url_25","2002-5-8"},
-			new Object [] {"url_41","2004-4-27"},
-			new Object [] {"url_17","2008-7-17"},
-			new Object [] {"url_26","2009-12-16"},
-			new Object [] {"url_34","2006-2-10"},
-			new Object [] {"url_8","2009-4-14"},
-			new Object [] {"url_16","2000-2-24"},
-			new Object [] {"url_2","2009-2-10"},
-			new Object [] {"url_35","2003-2-24"},
-			new Object [] {"url_34","2008-3-16"},
-			new Object [] {"url_27","2005-1-5"},
-			new Object [] {"url_8","2008-12-10"},
-			new Object [] {"url_38","2009-2-11"},
-			new Object [] {"url_38","2006-11-3"},
-			new Object [] {"url_47","2003-2-13"},
-			new Object [] {"url_8","2008-11-17"},
-			new Object [] {"url_26","2009-5-11"},
-			new Object [] {"url_12","2007-11-26"},
-			new Object [] {"url_10","2003-1-13"},
-			new Object [] {"url_8","2005-9-23"},
-			new Object [] {"url_42","2001-4-5"},
-			new Object [] {"url_30","2009-12-10"},
-			new Object [] {"url_2","2003-1-3"},
-			new Object [] {"url_2","2009-2-19"},
-			new Object [] {"url_7","2000-6-25"},
-			new Object [] {"url_15","2004-9-26"},
-			new Object [] {"url_25","2009-10-5"},
-			new Object [] {"url_23","2009-8-9"},
-			new Object [] {"url_27","2004-4-3"},
-			new Object [] {"url_37","2008-6-9"},
-			new Object [] {"url_9","2002-5-25"},
-			new Object [] {"url_43","2009-5-18"},
-			new Object [] {"url_21","2008-4-19"},
-			new Object [] {"url_12","2001-12-25"},
-			new Object [] {"url_16","2006-9-25"},
-			new Object [] {"url_27","2002-1-2"},
-			new Object [] {"url_2","2009-1-21"},
-			new Object [] {"url_31","2009-3-20"},
-			new Object [] {"url_42","2002-3-1"},
-			new Object [] {"url_31","2001-11-26"},
-			new Object [] {"url_20","2003-5-15"},
-			new Object [] {"url_32","2004-1-22"},
-			new Object [] {"url_28","2008-9-16"},
-			new Object [] {"url_27","2006-7-3"},
-			new Object [] {"url_11","2008-12-26"},
-			new Object [] {"url_15","2004-8-16"},
-			new Object [] {"url_34","2002-10-5"},
-			new Object [] {"url_44","2000-2-15"},
-			new Object [] {"url_9","2000-10-23"},
-			new Object [] {"url_45","2005-4-24"},
-			new Object [] {"url_0","2006-8-7"},
-			new Object [] {"url_48","2003-8-7"},
-			new Object [] {"url_8","2007-12-13"},
-			new Object [] {"url_42","2003-8-2"},
-			new Object [] {"url_25","2008-3-5"},
-			new Object [] {"url_3","2007-3-9"},
-			new Object [] {"url_49","2003-10-7"},
-			new Object [] {"url_18","2007-12-6"},
-			new Object [] {"url_3","2006-7-5"},
-			new Object [] {"url_27","2000-9-14"},
-			new Object [] {"url_42","2002-10-20"},
-			new Object [] {"url_44","2007-1-13"},
-			new Object [] {"url_6","2003-1-21"},
-			new Object [] {"url_40","2009-10-20"},
-			new Object [] {"url_28","2009-6-17"},
-			new Object [] {"url_22","2000-2-17"},
-			new Object [] {"url_3","2005-1-15"},
-			new Object [] {"url_9","2008-12-9"},
-			new Object [] {"url_9","2005-2-19"},
-			new Object [] {"url_28","2000-4-22"},
-			new Object [] {"url_44","2001-9-9"},
-			new Object [] {"url_43","2008-6-21"},
-			new Object [] {"url_39","2008-5-9"},
-			new Object [] {"url_15","2006-9-15"},
-			new Object [] {"url_23","2001-12-18"},
-			new Object [] {"url_14","2002-5-23"},
-			new Object [] {"url_11","2007-7-11"},
-			new Object [] {"url_34","2000-12-8"},
-			new Object [] {"url_47","2005-7-3"},
-			new Object [] {"url_38","2004-3-26"},
-			new Object [] {"url_19","2003-9-14"},
-			new Object [] {"url_24","2007-7-16"},
-			new Object [] {"url_40","2008-8-21"},
-			new Object [] {"url_17","2007-12-4"},
-			new Object [] {"url_25","2006-6-24"},
-			new Object [] {"url_2","2000-10-8"},
-			new Object [] {"url_12","2008-6-10"},
-			new Object [] {"url_11","2004-11-24"},
-			new Object [] {"url_13","2005-11-3"},
-			new Object [] {"url_43","2005-1-2"},
-			new Object [] {"url_14","2008-6-12"},
-			new Object [] {"url_43","2001-8-27"},
-			new Object [] {"url_45","2000-3-3"},
-			new Object [] {"url_0","2006-9-27"},
-			new Object [] {"url_22","2007-12-18"},
-			new Object [] {"url_25","2006-4-4"},
-			new Object [] {"url_32","2001-6-25"},
-			new Object [] {"url_6","2007-6-9"},
-			new Object [] {"url_8","2009-10-3"},
-			new Object [] {"url_15","2003-2-23"},
-			new Object [] {"url_37","2000-5-6"},
-			new Object [] {"url_27","2004-3-21"},
-			new Object [] {"url_17","2005-6-20"},
-			new Object [] {"url_2","2004-2-27"},
-			new Object [] {"url_36","2005-3-16"},
-			new Object [] {"url_1","2009-12-3"},
-			new Object [] {"url_9","2004-4-27"},
-			new Object [] {"url_18","2009-5-26"},
-			new Object [] {"url_31","2000-9-21"},
-			new Object [] {"url_12","2008-9-25"},
-			new Object [] {"url_2","2004-2-16"},
-			new Object [] {"url_28","2008-11-12"},
-			new Object [] {"url_28","2001-6-26"},
-			new Object [] {"url_12","2006-3-15"},
-			new Object [] {"url_0","2009-3-1"},
-			new Object [] {"url_36","2006-10-13"},
-			new Object [] {"url_15","2004-11-5"},
-			new Object [] {"url_32","2008-2-11"},
-			new Object [] {"url_19","2009-8-3"},
-			new Object [] {"url_2","2006-8-6"},
-			new Object [] {"url_11","2009-10-13"},
-			new Object [] {"url_21","2002-9-14"},
-			new Object [] {"url_18","2000-11-2"},
-			new Object [] {"url_35","2006-5-15"},
-			new Object [] {"url_11","2006-2-18"},
-			new Object [] {"url_0","2001-4-25"},
-			new Object [] {"url_14","2009-4-8"},
-			new Object [] {"url_16","2009-4-7"}
-	};
-
-	public static DataSet<Tuple2<String, String>> getDocumentDataSet(ExecutionEnvironment env) {
-
-		List<Tuple2<String, String>> data = new ArrayList<Tuple2<String, String>>(100);
-		for (Object [] document : DOCUMENTS) {
-			data.add(new Tuple2<String, String>((String) document[0], (String) document[1]));
-		}
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple3<Integer, String, Integer>> getRankDataSet(ExecutionEnvironment env) {
-
-		List<Tuple3<Integer, String, Integer>> data = new ArrayList<Tuple3<Integer, String, Integer>>(100);
-		for (Object [] rank : RANKS) {
-			data.add(new Tuple3<Integer, String, Integer>((Integer) rank[0], (String) rank[1], (Integer) rank[2]));
-		}
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple2<String, String>> getVisitDataSet(ExecutionEnvironment env) {
-
-		List<Tuple2<String, String>> data = new ArrayList<Tuple2<String, String>>(100);
-
-		for (Object [] visit : VISITS) {
-			data.add(new Tuple2<String, String>((String) visit[0], (String) visit[1]));
-		}
-		return env.fromCollection(data);
-
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogDataGenerator.java
deleted file mode 100644
index 5a8f0ac..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/util/WebLogDataGenerator.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.relational.util;
-
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.Calendar;
-import java.util.Random;
-
-import org.apache.flink.example.java.relational.WebLogAnalysis;
-
-/**
- * Data generator for the {@link WebLogAnalysis} example program. 
- *
- */
-public class WebLogDataGenerator {
-
-	/**
-	 * Main method to generate data for the {@link WebLogAnalysis} example program.
-	 * <p>
-	 * The generator creates to files:
-	 * <ul>
-	 * <li><code>{tmp.dir}/documents</code> for the web documents
-	 * <li><code>{tmp.dir}/ranks</code> for the ranks of the web documents
-	 * <li><code>{tmp.dir}/visits</code> for the logged visits of web documents
-	 * </ul> 
-	 * 
-	 * @param args 
-	 * <ol>
-	 * <li>Int: Number of web documents
-	 * <li>Int: Number of visits
-	 * </ol>
-	 */
-	public static void main(String[] args) {
-
-		// parse parameters
-		if (args.length < 2) {
-			System.out.println("WebLogDataGenerator <numberOfDocuments> <numberOfVisits>");
-			System.exit(1);
-		}
-		
-		int noDocs = Integer.parseInt(args[0]);
-		int noVisits = Integer.parseInt(args[1]);
-		
-		String[] filterKWs = { "editors", "oscillations", "convection" };
-
-		String[] words = { "Lorem", "ipsum", "dolor", "sit", "amet",
-				"consectetuer", "adipiscing", "elit", "sed", "diam", "nonummy",
-				"nibh", "euismod", "tincidunt", "ut", "laoreet", "dolore",
-				"magna", "aliquam", "erat", "volutpat", "Ut", "wisi", "enim",
-				"ad", "minim", "veniam", "quis", "nostrud", "exerci", "tation",
-				"ullamcorper", "suscipit", "lobortis", "nisl", "ut", "aliquip",
-				"ex", "ea", "commodo" };
-
-		
-		final String outPath = System.getProperty("java.io.tmpdir");
-
-		System.out.println("Generating documents files...");
-		genDocs(noDocs, filterKWs, words, outPath + "/documents");
-		System.out.println("Generating ranks files...");
-		genRanks(noDocs, outPath + "/ranks");
-		System.out.println("Generating visits files...");
-		genVisits(noVisits, noDocs, outPath + "/visits");
-
-		System.out.println("Done!");
-	}
-
-	/**
-	 * Generates the files for the documents relation. The entries apply the
-	 * following format: <br />
-	 * <code>URL | Content</code>
-	 * 
-	 * @param noDocs
-	 *            Number of entries for the documents relation
-	 * @param filterKeyWords
-	 *            A list of keywords that should be contained
-	 * @param words
-	 *            A list of words to fill the entries
-	 * @param path
-	 *            Output path for the documents relation
-	 */
-	private static void genDocs(int noDocs, String[] filterKeyWords, String[] words, String path) {
-
-		Random rand = new Random(Calendar.getInstance().getTimeInMillis());
-
-		try {
-			FileWriter fw = new FileWriter(path);
-
-			for (int i = 0; i < noDocs; i++) {
-
-				int wordsInDoc = rand.nextInt(40) + 10;
-				// URL
-				StringBuilder doc = new StringBuilder("url_" + i + "|");
-				for (int j = 0; j < wordsInDoc; j++) {
-					if (rand.nextDouble() > 0.9) {
-						// Approx. every 10th word is a keyword
-						doc.append(filterKeyWords[rand
-								.nextInt(filterKeyWords.length)] + " ");
-					} else {
-						// Fills up the docs file(s) with random words
-						doc.append(words[rand.nextInt(words.length)] + " ");
-					}
-				}
-				doc.append("|\n");
-
-				fw.write(doc.toString());
-			}
-			fw.close();
-
-		} catch (IOException e) {
-			e.printStackTrace();
-		}
-	}
-
-	/**
-	 * Generates the files for the ranks relation. The ranks entries apply the
-	 * following format: <br />
-	 * <code>Rank | URL | Average Duration |\n</code>
-	 * 
-	 * @param noDocs
-	 *            Number of entries in the documents relation
-	 * @param path
-	 *            Output path for the ranks relation
-	 */
-	private static void genRanks(int noDocs, String path) {
-
-		Random rand = new Random(Calendar.getInstance().getTimeInMillis());
-
-		try {
-			FileWriter fw = new FileWriter(path);
-
-			for (int i = 0; i < noDocs; i++) {
-				// Rank
-				StringBuilder rank = new StringBuilder(rand.nextInt(100) + "|");
-				// URL
-				rank.append("url_" + i + "|");
-				// Average duration
-				rank.append(rand.nextInt(10) + rand.nextInt(50) + "|\n");
-
-				fw.write(rank.toString());
-			}
-			fw.close();
-
-		} catch (IOException e) {
-			e.printStackTrace();
-		}
-	}
-
-	/**
-	 * Generates the files for the visits relation. The visits entries apply the
-	 * following format:<br />
-	 * <code>IP Address | URL | Date (YYYY-MM-DD) | Misc. Data (e.g. User-Agent) |\n</code>
-	 * 
-	 * @param noVisits
-	 *            Number of entries for the visits relation
-	 * @param noDocs
-	 *            Number of entries in the documents relation
-	 * @param path
-	 *            Output path for the visits relation
-	 */
-	private static void genVisits(int noVisits, int noDocs, String path) {
-
-		Random rand = new Random(Calendar.getInstance().getTimeInMillis());
-
-		try {
-			FileWriter fw = new FileWriter(path);
-
-			for (int i = 0; i < noVisits; i++) {
-
-				int year = 2000 + rand.nextInt(10); // yearFilter 3
-				int month = rand.nextInt(12) + 1; // month between 1 and 12
-				int day = rand.nextInt(27) + 1; // day between 1 and 28
-
-				// IP address
-				StringBuilder visit = new StringBuilder(rand.nextInt(256) + "."
-						+ rand.nextInt(256) + "." + rand.nextInt(256) + "."
-						+ rand.nextInt(256) + "|");
-				// URL
-				visit.append("url_" + rand.nextInt(noDocs) + "|");
-				// Date (format: YYYY-MM-DD)
-				visit.append(year + "-" + month + "-" + day + "|");
-				// Miscellaneous data, e.g. User-Agent
-				visit.append("0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n");
-
-				fw.write(visit.toString());
-			}
-			fw.close();
-
-		} catch (IOException e) {
-			e.printStackTrace();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java
deleted file mode 100644
index 3e95ccd..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/WordCount.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.wordcount;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.wordcount.util.WordCountData;
-
-/**
- * Implements the "WordCount" program that computes a simple word occurrence histogram
- * over text files. 
- * 
- * <p>
- * The input is a plain text file with lines separated by newline characters.
- * 
- * <p>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * 
- * <p>
- * This example shows how to:
- * <ul>
- * <li>write a simple Flink program.
- * <li>use Tuple data types.
- * <li>write and use user-defined functions. 
- * </ul>
- * 
- */
-@SuppressWarnings("serial")
-public class WordCount {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		// set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// get input data
-		DataSet<String> text = getTextDataSet(env);
-		
-		DataSet<Tuple2<String, Integer>> counts = 
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				text.flatMap(new Tokenizer())
-				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0)
-				.sum(1);
-
-		// emit result
-		if(fileOutput) {
-			counts.writeAsCsv(outputPath, "\n", " ");
-		} else {
-			counts.print();
-		}
-		
-		// execute program
-		env.execute("WordCount Example");
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-	
-	/**
-	 * Implements the string tokenizer that splits sentences into words as a user-defined
-	 * FlatMapFunction. The function takes a line (String) and splits it into 
-	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
-	 */
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-			
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-	
-	private static boolean parseParameters(String[] args) {
-		
-		if(args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: WordCount <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing WordCount example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from a file.");
-			System.out.println("  Usage: WordCount <text path> <result path>");
-		}
-		return true;
-	}
-	
-	private static DataSet<String> getTextDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		} else {
-			// get default test text data
-			return WordCountData.getDefaultTextLineDataSet(env);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
deleted file mode 100644
index b7ee4b5..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.wordcount.util;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * Provides the default data sets used for the WordCount example program.
- * The default data sets are used, if no parameters are given to the program.
- *
- */
-public class WordCountData {
-
-	public static final String[] WORDS = new String[] {
-		"To be, or not to be,--that is the question:--",
-		"Whether 'tis nobler in the mind to suffer",
-		"The slings and arrows of outrageous fortune",
-		"Or to take arms against a sea of troubles,",
-		"And by opposing end them?--To die,--to sleep,--",
-		"No more; and by a sleep to say we end",
-		"The heartache, and the thousand natural shocks",
-		"That flesh is heir to,--'tis a consummation",
-		"Devoutly to be wish'd. To die,--to sleep;--",
-		"To sleep! perchance to dream:--ay, there's the rub;",
-		"For in that sleep of death what dreams may come,",
-		"When we have shuffled off this mortal coil,",
-		"Must give us pause: there's the respect",
-		"That makes calamity of so long life;",
-		"For who would bear the whips and scorns of time,",
-		"The oppressor's wrong, the proud man's contumely,",
-		"The pangs of despis'd love, the law's delay,",
-		"The insolence of office, and the spurns",
-		"That patient merit of the unworthy takes,",
-		"When he himself might his quietus make",
-		"With a bare bodkin? who would these fardels bear,",
-		"To grunt and sweat under a weary life,",
-		"But that the dread of something after death,--",
-		"The undiscover'd country, from whose bourn",
-		"No traveller returns,--puzzles the will,",
-		"And makes us rather bear those ills we have",
-		"Than fly to others that we know not of?",
-		"Thus conscience does make cowards of us all;",
-		"And thus the native hue of resolution",
-		"Is sicklied o'er with the pale cast of thought;",
-		"And enterprises of great pith and moment,",
-		"With this regard, their currents turn awry,",
-		"And lose the name of action.--Soft you now!",
-		"The fair Ophelia!--Nymph, in thy orisons",
-		"Be all my sins remember'd."
-	};
-
-	public static DataSet<String> getDefaultTextLineDataSet(ExecutionEnvironment env) {
-		return env.fromElements(WORDS);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
new file mode 100644
index 0000000..1ba05a4
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
@@ -0,0 +1,337 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.java.clustering;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.examples.java.clustering.util.KMeansData;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+
+/**
+ * This example implements a basic K-Means clustering algorithm.
+ * 
+ * <p>
+ * K-Means is an iterative clustering algorithm and works as follows:<br>
+ * K-Means is given a set of data points to be clustered and an initial set of <i>K</i> cluster centers.
+ * In each iteration, the algorithm computes the distance of each data point to each cluster center.
+ * Each point is assigned to the cluster center which is closest to it.
+ * Subsequently, each cluster center is moved to the center (<i>mean</i>) of all points that have been assigned to it.
+ * The moved cluster centers are fed into the next iteration. 
+ * The algorithm terminates after a fixed number of iterations (as in this implementation) 
+ * or if cluster centers do not (significantly) move in an iteration.<br>
+ * This is the Wikipedia entry for the <a href="http://en.wikipedia.org/wiki/K-means_clustering">K-Means Clustering algorithm</a>.
+ * 
+ * <p>
+ * This implementation works on two-dimensional data points. <br>
+ * It computes an assignment of data points to cluster centers, i.e., 
+ * each data point is annotated with the id of the final cluster (center) it belongs to.
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Data points are represented as two double values separated by a blank character.
+ * Data points are separated by newline characters.<br>
+ * For example <code>"1.2 2.3\n5.3 7.2\n"</code> gives two data points (x=1.2, y=2.3) and (x=5.3, y=7.2).
+ * <li>Cluster centers are represented by an integer id and a point value.<br>
+ * For example <code>"1 6.2 3.2\n2 2.9 5.7\n"</code> gives two centers (id=1, x=6.2, y=3.2) and (id=2, x=2.9, y=5.7).
+ * </ul>
+ * 
+ * <p>
+ * Usage: <code>KMeans &lt;points path&gt; &lt;centers path&gt; &lt;result path&gt; &lt;num iterations&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link KMeansData} and 10 iterations. 
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Bulk iterations
+ * <li>Broadcast variables in bulk iterations
+ * <li>Custom Java objects (PoJos)
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class KMeans {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String[] args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+	
+		// set up execution environment
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// get input data
+		DataSet<Point> points = getPointDataSet(env);
+		DataSet<Centroid> centroids = getCentroidDataSet(env);
+		
+		// set number of bulk iterations for KMeans algorithm
+		IterativeDataSet<Centroid> loop = centroids.iterate(numIterations);
+		
+		DataSet<Centroid> newCentroids = points
+			// compute closest centroid for each point
+			.map(new SelectNearestCenter()).withBroadcastSet(loop, "centroids")
+			// count and sum point coordinates for each centroid
+			.map(new CountAppender())
+			.groupBy(0).reduce(new CentroidAccumulator())
+			// compute new centroids from point counts and coordinate sums
+			.map(new CentroidAverager());
+		
+		// feed new centroids back into next iteration
+		DataSet<Centroid> finalCentroids = loop.closeWith(newCentroids);
+		
+		DataSet<Tuple2<Integer, Point>> clusteredPoints = points
+				// assign points to final clusters
+				.map(new SelectNearestCenter()).withBroadcastSet(finalCentroids, "centroids");
+		
+		// emit result
+		if(fileOutput) {
+			clusteredPoints.writeAsCsv(outputPath, "\n", " ");
+		} else {
+			clusteredPoints.print();
+		}
+
+		// execute program
+		env.execute("KMeans Example");
+		
+	}
+	
+	// *************************************************************************
+	//     DATA TYPES
+	// *************************************************************************
+	
+	/**
+	 * A simple two-dimensional point.
+	 */
+	public static class Point implements Serializable {
+		
+		public double x, y;
+		
+		public Point() {}
+
+		public Point(double x, double y) {
+			this.x = x;
+			this.y = y;
+		}
+		
+		public Point add(Point other) {
+			x += other.x;
+			y += other.y;
+			return this;
+		}
+		
+		public Point div(long val) {
+			x /= val;
+			y /= val;
+			return this;
+		}
+		
+		public double euclideanDistance(Point other) {
+			return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y));
+		}
+		
+		public void clear() {
+			x = y = 0.0;
+		}
+		
+		@Override
+		public String toString() {
+			return x + " " + y;
+		}
+	}
+	
+	/**
+	 * A simple two-dimensional centroid, basically a point with an ID. 
+	 */
+	public static class Centroid extends Point {
+		
+		public int id;
+		
+		public Centroid() {}
+		
+		public Centroid(int id, double x, double y) {
+			super(x,y);
+			this.id = id;
+		}
+		
+		public Centroid(int id, Point p) {
+			super(p.x, p.y);
+			this.id = id;
+		}
+		
+		@Override
+		public String toString() {
+			return id + " " + super.toString();
+		}
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+	
+	/** Converts a Tuple2<Double,Double> into a Point. */
+	public static final class TuplePointConverter implements MapFunction<Tuple2<Double, Double>, Point> {
+
+		@Override
+		public Point map(Tuple2<Double, Double> t) throws Exception {
+			return new Point(t.f0, t.f1);
+		}
+	}
+	
+	/** Converts a Tuple3<Integer, Double,Double> into a Centroid. */
+	public static final class TupleCentroidConverter implements MapFunction<Tuple3<Integer, Double, Double>, Centroid> {
+
+		@Override
+		public Centroid map(Tuple3<Integer, Double, Double> t) throws Exception {
+			return new Centroid(t.f0, t.f1, t.f2);
+		}
+	}
+	
+	/** Determines the closest cluster center for a data point. */
+	public static final class SelectNearestCenter extends RichMapFunction<Point, Tuple2<Integer, Point>> {
+		private Collection<Centroid> centroids;
+
+		/** Reads the centroid values from a broadcast variable into a collection. */
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			this.centroids = getRuntimeContext().getBroadcastVariable("centroids");
+		}
+		
+		@Override
+		public Tuple2<Integer, Point> map(Point p) throws Exception {
+			
+			double minDistance = Double.MAX_VALUE;
+			int closestCentroidId = -1;
+			
+			// check all cluster centers
+			for (Centroid centroid : centroids) {
+				// compute distance
+				double distance = p.euclideanDistance(centroid);
+				
+				// update nearest cluster if necessary 
+				if (distance < minDistance) {
+					minDistance = distance;
+					closestCentroidId = centroid.id;
+				}
+			}
+
+			// emit a new record with the center id and the data point.
+			return new Tuple2<Integer, Point>(closestCentroidId, p);
+		}
+	}
+	
+	/** Appends a count variable to the tuple. */ 
+	public static final class CountAppender implements MapFunction<Tuple2<Integer, Point>, Tuple3<Integer, Point, Long>> {
+
+		@Override
+		public Tuple3<Integer, Point, Long> map(Tuple2<Integer, Point> t) {
+			return new Tuple3<Integer, Point, Long>(t.f0, t.f1, 1L);
+		} 
+	}
+	
+	/** Sums and counts point coordinates. */
+	public static final class CentroidAccumulator implements ReduceFunction<Tuple3<Integer, Point, Long>> {
+
+		@Override
+		public Tuple3<Integer, Point, Long> reduce(Tuple3<Integer, Point, Long> val1, Tuple3<Integer, Point, Long> val2) {
+			return new Tuple3<Integer, Point, Long>(val1.f0, val1.f1.add(val2.f1), val1.f2 + val2.f2);
+		}
+	}
+	
+	/** Computes new centroid from coordinate sum and count of points. */
+	public static final class CentroidAverager implements MapFunction<Tuple3<Integer, Point, Long>, Centroid> {
+
+		@Override
+		public Centroid map(Tuple3<Integer, Point, Long> value) {
+			return new Centroid(value.f0, value.f1.div(value.f2));
+		}
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean fileOutput = false;
+	private static String pointsPath = null;
+	private static String centersPath = null;
+	private static String outputPath = null;
+	private static int numIterations = 10;
+	
+	private static boolean parseParameters(String[] programArguments) {
+		
+		if(programArguments.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(programArguments.length == 4) {
+				pointsPath = programArguments[0];
+				centersPath = programArguments[1];
+				outputPath = programArguments[2];
+				numIterations = Integer.parseInt(programArguments[3]);
+			} else {
+				System.err.println("Usage: KMeans <points path> <centers path> <result path> <num iterations>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing K-Means example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  We provide a data generator to create synthetic input files for this program.");
+			System.out.println("  Usage: KMeans <points path> <centers path> <result path> <num iterations>");
+		}
+		return true;
+	}
+	
+	private static DataSet<Point> getPointDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			// read points from CSV file
+			return env.readCsvFile(pointsPath)
+						.fieldDelimiter(' ')
+						.includeFields(true, true)
+						.types(Double.class, Double.class)
+						.map(new TuplePointConverter());
+		} else {
+			return KMeansData.getDefaultPointDataSet(env);
+		}
+	}
+	
+	private static DataSet<Centroid> getCentroidDataSet(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return env.readCsvFile(centersPath)
+						.fieldDelimiter(' ')
+						.includeFields(true, true, true)
+						.types(Integer.class, Double.class, Double.class)
+						.map(new TupleCentroidConverter());
+		} else {
+			return KMeansData.getDefaultCentroidDataSet(env);
+		}
+	}
+		
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java
new file mode 100644
index 0000000..233408b
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.clustering.util;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.clustering.KMeans.Centroid;
+import org.apache.flink.examples.java.clustering.KMeans.Point;
+
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the K-Means example program.
+ * The default data sets are used, if no parameters are given to the program.
+ *
+ */
+public class KMeansData {
+
+	// We have the data as object arrays so that we can also generate Scala Data Sources from it.
+	public static final Object[][] CENTROIDS = new Object[][] {
+		new Object[] {1, -31.85, -44.77},
+		new Object[]{2, 35.16, 17.46},
+		new Object[]{3, -5.16, 21.93},
+		new Object[]{4, -24.06, 6.81}
+	};
+
+	public static final Object[][] POINTS = new Object[][] {
+		new Object[] {-14.22, -48.01},
+		new Object[] {-22.78, 37.10},
+		new Object[] {56.18, -42.99},
+		new Object[] {35.04, 50.29},
+		new Object[] {-9.53, -46.26},
+		new Object[] {-34.35, 48.25},
+		new Object[] {55.82, -57.49},
+		new Object[] {21.03, 54.64},
+		new Object[] {-13.63, -42.26},
+		new Object[] {-36.57, 32.63},
+		new Object[] {50.65, -52.40},
+		new Object[] {24.48, 34.04},
+		new Object[] {-2.69, -36.02},
+		new Object[] {-38.80, 36.58},
+		new Object[] {24.00, -53.74},
+		new Object[] {32.41, 24.96},
+		new Object[] {-4.32, -56.92},
+		new Object[] {-22.68, 29.42},
+		new Object[] {59.02, -39.56},
+		new Object[] {24.47, 45.07},
+		new Object[] {5.23, -41.20},
+		new Object[] {-23.00, 38.15},
+		new Object[] {44.55, -51.50},
+		new Object[] {14.62, 59.06},
+		new Object[] {7.41, -56.05},
+		new Object[] {-26.63, 28.97},
+		new Object[] {47.37, -44.72},
+		new Object[] {29.07, 51.06},
+		new Object[] {0.59, -31.89},
+		new Object[] {-39.09, 20.78},
+		new Object[] {42.97, -48.98},
+		new Object[] {34.36, 49.08},
+		new Object[] {-21.91, -49.01},
+		new Object[] {-46.68, 46.04},
+		new Object[] {48.52, -43.67},
+		new Object[] {30.05, 49.25},
+		new Object[] {4.03, -43.56},
+		new Object[] {-37.85, 41.72},
+		new Object[] {38.24, -48.32},
+		new Object[] {20.83, 57.85}
+	};
+
+	public static DataSet<Centroid> getDefaultCentroidDataSet(ExecutionEnvironment env) {
+		List<Centroid> centroidList = new LinkedList<Centroid>();
+		for (Object[] centroid : CENTROIDS) {
+			centroidList.add(
+					new Centroid((Integer) centroid[0], (Double) centroid[1], (Double) centroid[2]));
+		}
+		return env.fromCollection(centroidList);
+	}
+	
+	public static DataSet<Point> getDefaultPointDataSet(ExecutionEnvironment env) {
+		List<Point> pointList = new LinkedList<Point>();
+		for (Object[] point : POINTS) {
+			pointList.add(new Point((Double) point[0], (Double) point[1]));
+		}
+		return env.fromCollection(pointList);
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
new file mode 100644
index 0000000..0d94e77
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.clustering.util;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.Locale;
+import java.util.Random;
+
+import org.apache.flink.examples.java.clustering.KMeans;
+
+/**
+ * Generates data for the {@link KMeans} example program.
+ */
+public class KMeansDataGenerator {
+	
+	static {
+		Locale.setDefault(Locale.US);
+	}
+	
+	private static final String CENTERS_FILE = "centers";
+	private static final String POINTS_FILE = "points";
+	private static final long DEFAULT_SEED = 4650285087650871364L;
+	private static final double DEFAULT_VALUE_RANGE = 100.0;
+	private static final double RELATIVE_STDDEV = 0.08;
+	private static final int DIMENSIONALITY = 2;
+	private static final DecimalFormat FORMAT = new DecimalFormat("#0.00");
+	private static final char DELIMITER = ' ';
+
+	/**
+	 * Main method to generate data for the {@link KMeans} example program.
+	 * <p>
+	 * The generator creates to files:
+	 * <ul>
+	 * <li><code>{tmp.dir}/points</code> for the data points
+	 * <li><code>{tmp.dir}/centers</code> for the cluster centers
+	 * </ul> 
+	 * 
+	 * @param args 
+	 * <ol>
+	 * <li>Int: Number of data points
+	 * <li>Int: Number of cluster centers
+	 * <li><b>Optional</b> Double: Standard deviation of data points
+	 * <li><b>Optional</b> Double: Value range of cluster centers
+	 * <li><b>Optional</b> Long: Random seed
+	 * </ol>
+	 */
+	public static void main(String[] args) throws IOException {
+
+		// check parameter count
+		if (args.length < 2) {
+			System.out.println("KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]");
+			System.exit(1);
+		}
+
+		// parse parameters
+		final int numDataPoints = Integer.parseInt(args[0]);
+		final int k = Integer.parseInt(args[1]);
+		final double stddev = args.length > 2 ? Double.parseDouble(args[2]) : RELATIVE_STDDEV;
+		final double range = args.length > 3 ? Double.parseDouble(args[4]) : DEFAULT_VALUE_RANGE;
+		final long firstSeed = args.length > 4 ? Long.parseLong(args[4]) : DEFAULT_SEED;
+		
+		final double absoluteStdDev = stddev * range;
+		final Random random = new Random(firstSeed);
+		final String tmpDir = System.getProperty("java.io.tmpdir");
+		
+		// the means around which data points are distributed
+		final double[][] means = uniformRandomCenters(random, k, DIMENSIONALITY, range);
+		
+		// write the points out
+		BufferedWriter pointsOut = null;
+		try {
+			pointsOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+POINTS_FILE)));
+			StringBuilder buffer = new StringBuilder();
+			
+			double[] point = new double[DIMENSIONALITY];
+			int nextCentroid = 0;
+			
+			for (int i = 1; i <= numDataPoints; i++) {
+				// generate a point for the current centroid
+				double[] centroid = means[nextCentroid];
+				for (int d = 0; d < DIMENSIONALITY; d++) {
+					point[d] = (random.nextGaussian() * absoluteStdDev) + centroid[d];
+				}
+				writePoint(point, buffer, pointsOut);
+				nextCentroid = (nextCentroid + 1) % k;
+			}
+		}
+		finally {
+			if (pointsOut != null) {
+				pointsOut.close();
+			}
+		}
+		
+		// write the uniformly distributed centers to a file
+		BufferedWriter centersOut = null;
+		try {
+			centersOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+CENTERS_FILE)));
+			StringBuilder buffer = new StringBuilder();
+			
+			double[][] centers = uniformRandomCenters(random, k, DIMENSIONALITY, range);
+			
+			for (int i = 0; i < k; i++) {
+				writeCenter(i + 1, centers[i], buffer, centersOut);
+			}
+		}
+		finally {
+			if (centersOut != null) {
+				centersOut.close();
+			}
+		}
+		
+		System.out.println("Wrote "+numDataPoints+" data points to "+tmpDir+"/"+POINTS_FILE);
+		System.out.println("Wrote "+k+" cluster centers to "+tmpDir+"/"+CENTERS_FILE);
+	}
+	
+	private static final double[][] uniformRandomCenters(Random rnd, int num, int dimensionality, double range) {
+		final double halfRange = range / 2;
+		final double[][] points = new double[num][dimensionality];
+		
+		for (int i = 0; i < num; i++) {
+			for (int dim = 0; dim < dimensionality; dim ++) {
+				points[i][dim] = (rnd.nextDouble() * range) - halfRange;
+			}
+		}
+		return points;
+	}
+	
+	private static void writePoint(double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException {
+		buffer.setLength(0);
+		
+		// write coordinates
+		for (int j = 0; j < coordinates.length; j++) {
+			buffer.append(FORMAT.format(coordinates[j]));
+			if(j < coordinates.length - 1) {
+				buffer.append(DELIMITER);
+			}
+		}
+		
+		out.write(buffer.toString());
+		out.newLine();
+	}
+	
+	private static void writeCenter(long id, double[] coordinates, StringBuilder buffer, BufferedWriter out) throws IOException {
+		buffer.setLength(0);
+		
+		// write id
+		buffer.append(id);
+		buffer.append(DELIMITER);
+
+		// write coordinates
+		for (int j = 0; j < coordinates.length; j++) {
+			buffer.append(FORMAT.format(coordinates[j]));
+			if(j < coordinates.length - 1) {
+				buffer.append(DELIMITER);
+			}
+		}
+		
+		out.write(buffer.toString());
+		out.newLine();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java
new file mode 100644
index 0000000..7312dd8
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.examples.java.graph;
+
+import org.apache.flink.api.common.ProgramDescription;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFields;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsFirst;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ConstantFieldsSecond;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.examples.java.graph.util.ConnectedComponentsData;
+
+/**
+ * An implementation of the connected components algorithm, using a delta iteration.
+ * 
+ * <p>
+ * Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the minimum of its own ID and its
+ * neighbors' IDs, as its new ID and tells its neighbors about its new ID. After the algorithm has completed, all vertices in the
+ * same component will have the same ID.
+ * 
+ * <p>
+ * A vertex whose component ID did not change needs not propagate its information in the next step. Because of that,
+ * the algorithm is easily expressible via a delta iteration. We here model the solution set as the vertices with
+ * their current component ids, and the workset as the changed vertices. Because we see all vertices initially as
+ * changed, the initial workset and the initial solution set are identical. Also, the delta to the solution set
+ * is consequently also the next workset.<br>
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Vertices represented as IDs and separated by new-line characters.<br> 
+ * For example <code>"1\n2\n12\n42\n63\n"</code> gives five vertices (1), (2), (12), (42), and (63). 
+ * <li>Edges are represented as pairs for vertex IDs which are separated by space 
+ * characters. Edges are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+ * </ul>
+ * 
+ * <p>
+ * Usage: <code>ConnectedComponents &lt;vertices path&gt; &lt;edges path&gt; &lt;result path&gt; &lt;max number of iterations&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link ConnectedComponentsData} and 10 iterations. 
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Delta Iterations
+ * <li>Generic-typed Functions 
+ * </ul>
+ */
+@SuppressWarnings("serial")
+public class ConnectedComponents implements ProgramDescription {
+	
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+	
+	public static void main(String... args) throws Exception {
+		
+		if(!parseParameters(args)) {
+			return;
+		}
+		
+		// set up execution environment
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		// read vertex and edge data
+		DataSet<Long> vertices = getVertexDataSet(env);
+		DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
+		
+		// assign the initial components (equal to the vertex id)
+		DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
+				
+		// open a delta iteration
+		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
+				verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0);
+		
+		// apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller
+		DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset().join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
+				.groupBy(0).aggregate(Aggregations.MIN, 1)
+				.join(iteration.getSolutionSet()).where(0).equalTo(0)
+				.with(new ComponentIdFilter());
+
+		// close the delta iteration (delta and new workset are identical)
+		DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
+		
+		// emit result
+		if(fileOutput) {
+			result.writeAsCsv(outputPath, "\n", " ");
+		} else {
+			result.print();
+		}
+		
+		// execute program
+		env.execute("Connected Components Example");
+	}
+	
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+	
+	/**
+	 * Function that turns a value into a 2-tuple where both fields are that value.
+	 */
+	@ConstantFields("0 -> 0,1") 
+	public static final class DuplicateValue<T> implements MapFunction<T, Tuple2<T, T>> {
+		
+		@Override
+		public Tuple2<T, T> map(T vertex) {
+			return new Tuple2<T, T>(vertex, vertex);
+		}
+	}
+	
+	/**
+	 * Undirected edges by emitting for each input edge the input edges itself and an inverted version.
+	 */
+	public static final class UndirectEdge implements FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+		Tuple2<Long, Long> invertedEdge = new Tuple2<Long, Long>();
+		
+		@Override
+		public void flatMap(Tuple2<Long, Long> edge, Collector<Tuple2<Long, Long>> out) {
+			invertedEdge.f0 = edge.f1;
+			invertedEdge.f1 = edge.f0;
+			out.collect(edge);
+			out.collect(invertedEdge);
+		}
+	}
+	
+	/**
+	 * UDF that joins a (Vertex-ID, Component-ID) pair that represents the current component that
+	 * a vertex is associated with, with a (Source-Vertex-ID, Target-VertexID) edge. The function
+	 * produces a (Target-vertex-ID, Component-ID) pair.
+	 */
+	@ConstantFieldsFirst("1 -> 0")
+	@ConstantFieldsSecond("1 -> 1")
+	public static final class NeighborWithComponentIDJoin implements JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		@Override
+		public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithComponent, Tuple2<Long, Long> edge) {
+			return new Tuple2<Long, Long>(edge.f1, vertexWithComponent.f1);
+		}
+	}
+	
+
+
+	@ConstantFieldsFirst("0")
+	public static final class ComponentIdFilter implements FlatJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		@Override
+		public void join(Tuple2<Long, Long> candidate, Tuple2<Long, Long> old, Collector<Tuple2<Long, Long>> out) {
+			if (candidate.f1 < old.f1) {
+				out.collect(candidate);
+			}
+		}
+	}
+
+
+
+	@Override
+	public String getDescription() {
+		return "Parameters: <vertices-path> <edges-path> <result-path> <max-number-of-iterations>";
+	}
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+	
+	private static boolean fileOutput = false;
+	private static String verticesPath = null;
+	private static String edgesPath = null;
+	private static String outputPath = null;
+	private static int maxIterations = 10;
+	
+	private static boolean parseParameters(String[] programArguments) {
+		
+		if(programArguments.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if(programArguments.length == 4) {
+				verticesPath = programArguments[0];
+				edgesPath = programArguments[1];
+				outputPath = programArguments[2];
+				maxIterations = Integer.parseInt(programArguments[3]);
+			} else {
+				System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing Connected Components example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>");
+		}
+		return true;
+	}
+	
+	private static DataSet<Long> getVertexDataSet(ExecutionEnvironment env) {
+		
+		if(fileOutput) {
+			return env.readCsvFile(verticesPath).types(Long.class)
+						.map(
+								new MapFunction<Tuple1<Long>, Long>() {
+									public Long map(Tuple1<Long> value) { return value.f0; }
+								});
+		} else {
+			return ConnectedComponentsData.getDefaultVertexDataSet(env);
+		}
+	}
+	
+	private static DataSet<Tuple2<Long, Long>> getEdgeDataSet(ExecutionEnvironment env) {
+		
+		if(fileOutput) {
+			return env.readCsvFile(edgesPath).fieldDelimiter(' ').types(Long.class, Long.class); 
+		} else {
+			return ConnectedComponentsData.getDefaultEdgeDataSet(env);
+		}
+	}
+	
+	
+}


[39/60] git commit: Add ExecutionEnvironment to ScalaAPICompletessTest

Posted by al...@apache.org.
Add ExecutionEnvironment to ScalaAPICompletessTest


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

Branch: refs/heads/master
Commit: feade0592f4351b39f1cb02d8ae36d5364e2acd4
Parents: 4ddc3f7
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Sep 18 18:00:16 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/api/scala/ScalaAPICompletenessTest.scala  | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/feade059/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
index 5c4606f..97939f0 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
@@ -54,6 +54,8 @@ class ScalaAPICompletenessTest {
       "org.apache.flink.api.java.operators.TwoInputOperator.getInput2",
       "org.apache.flink.api.java.operators.TwoInputOperator.getInput1Type",
       "org.apache.flink.api.java.operators.TwoInputOperator.getInput2Type",
+      "org.apache.flink.api.java.ExecutionEnvironment.localExecutionIsAllowed",
+      "org.apache.flink.api.java.ExecutionEnvironment.setDefaultLocalParallelism",
 
       // This is really just a mapper, which in Scala can easily expressed as a map lambda
       "org.apache.flink.api.java.DataSet.writeAsFormattedText",
@@ -138,6 +140,10 @@ class ScalaAPICompletenessTest {
   def testCompleteness(): Unit = {
     checkMethods("DataSet", "DataSet", classOf[JavaDataSet[_]], classOf[DataSet[_]])
 
+    checkMethods(
+      "ExecutionEnvironment", "ExecutionEnvironment",
+      classOf[org.apache.flink.api.java.ExecutionEnvironment], classOf[ExecutionEnvironment])
+
     checkMethods("Operator", "DataSet", classOf[Operator[_, _]], classOf[DataSet[_]])
 
     checkMethods("UnsortedGrouping", "GroupedDataSet",


[46/60] [doc] Switch parser to kramdown, normalize Headings

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/programming_guide.md
----------------------------------------------------------------------
diff --git a/docs/programming_guide.md b/docs/programming_guide.md
new file mode 100644
index 0000000..79a4820
--- /dev/null
+++ b/docs/programming_guide.md
@@ -0,0 +1,1522 @@
+---
+title: "Flink Programming Guide"
+---
+
+* This will be replaced by the TOC
+{:toc}
+
+
+<a href="#top"></a>
+
+Introduction
+------------
+
+Analysis programs in Flink are regular Java programs that implement transformations on data sets
+(e.g., filtering, mapping, joining, grouping). The data sets are initially created from certain
+sources (e.g., by reading files, or from collections). Results are returned via sinks, which may for
+example write the data to (distributed) files, or to standard output (for example the command line
+terminal). Flink programs run in a variety of contexts, standalone, or embedded in other programs.
+The execution can happen in a local JVM, or on clusters of many machines.
+
+In order to create your own Flink program, we encourage you to start with the
+[program skeleton](#skeleton) and gradually add your own [transformations](#transformations).
+The remaining sections act as references for additional operations and advanced features.
+
+
+Example Program
+---------------
+
+The following program is a complete, working example of WordCount. You can copy &amp; paste the code
+to run it locally. You only have to include Flink's Java API library into your project (see Section
+[Linking with Flink](#linking)) and specify the imports. Then you are ready to go!
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+{% highlight java %}
+public class WordCountExample {
+    public static void main(String[] args) throws Exception {
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+	    DataSet<String> text = env.fromElements(
+            "Who's there?",
+            "I think I hear them. Stand, ho! Who's there?");
+
+        DataSet<Tuple2<String, Integer>> wordCounts = text
+            .flatMap(new LineSplitter())
+            .groupBy(0)
+            .sum(1);
+
+        wordCounts.print();
+
+        env.execute("Word Count Example");
+    }
+
+    public static class LineSplitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
+        @Override
+        public void flatMap(String line, Collector<Tuple2<String, Integer>> out) {
+            for (String word : line.split(" ")) {
+                out.collect(new Tuple2<String, Integer>(word, 1));
+            }
+        }
+    }
+}
+{% endhighlight %}
+
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+object WordCount {
+  def main(args: Array[String]) {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val text = env.fromElements("Who's there?",
+            "I think I hear them. Stand, ho! Who's there?")
+
+    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
+      .map { (_, 1) }
+      .groupBy(0)
+      .sum(1)
+
+    counts.print()
+
+    env.execute("Scala WordCount Example")
+  }
+}
+{% endhighlight %}
+</div>
+
+</div>
+
+[Back to top](#top)
+
+
+Linking with Flink
+------------------
+
+To write programs with Flink, you need to include Flink’s Java API library in your project.
+
+The simplest way to do this is to use the [quickstart scripts](java_api_quickstart.html). They
+create a blank project from a template (a Maven Archetype), which sets up everything for you. To
+manually create the project, you can use the archetype and create a project by calling:
+
+{% highlight bash %}
+mvn archetype:generate /
+    -DarchetypeGroupId=org.apache.flink/
+    -DarchetypeArtifactId=flink-quickstart-java /
+    -DarchetypeVersion={{site.FLINK_VERSION_STABLE }}
+{% endhighlight %}
+
+If you want to add Flink to an existing Maven project, add the following entry to your
+*dependencies* section in the *pom.xml* file of your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-java</artifactId>
+  <version>{{site.FLINK_VERSION_STABLE }}</version>
+</dependency>
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients</artifactId>
+  <version>{{site.FLINK_VERSION_STABLE }}</version>
+</dependency>
+{% endhighlight %}
+
+If you are using Flink together with Hadoop, the version of the dependency may vary depending on the
+version of Hadoop (or more specifically, HDFS) that you want to use Flink with. Please refer to the
+[downloads page]({{site.baseurl}}/downloads.html) for a list of available versions, and instructions
+on how to link with custom versions of Hadoop.
+
+In order to link against the latest SNAPSHOT versions of the code, please follow
+[this guide]({{site.baseurl}}/downloads.html#nightly).
+
+The *flink-clients* dependency is only necessary to invoke the Flink program locally (for example to
+run it standalone for testing and debugging).  If you intend to only export the program as a JAR
+file and [run it on a cluster](cluster_execution.html), you can skip that dependency.
+
+[Back to top](#top)
+
+Program Skeleton
+----------------
+
+As we already saw in the example, Flink programs look like regular Java
+programs with a `main()` method. Each program consists of the same basic parts:
+
+1. Obtain an `ExecutionEnvironment`,
+2. Load/create the initial data,
+3. Specify transformations on this data,
+4. Specify where to put the results of your computations, and
+5. Execute your program.
+
+We will now give an overview of each of those steps but please refer to the respective sections for
+more details. Note that all
+{% gh_link /flink-java/src/main/java/org/apache/flink/api/java "core classes of the Java API" %}
+are found in the package `org.apache.flink.api.java`.
+
+The `ExecutionEnvironment` is the basis for all Flink programs. You can
+obtain one using these static methods on class `ExecutionEnvironment`:
+
+{% highlight java %}
+getExecutionEnvironment()
+
+createLocalEnvironment()
+createLocalEnvironment(int degreeOfParallelism)
+
+createRemoteEnvironment(String host, int port, String... jarFiles)
+createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles)
+{% endhighlight %}
+
+Typically, you only need to use `getExecutionEnvironment()`, since this
+will do the right thing depending on the context: if you are executing
+your program inside an IDE or as a regular Java program it will create
+a local environment that will execute your program on your local machine. If
+you created a JAR file from you program, and invoke it through the [command line](cli.html)
+or the [web interface](web_client.html),
+the Flink cluster manager will
+execute your main method and `getExecutionEnvironment()` will return
+an execution environment for executing your program on a cluster.
+
+For specifying data sources the execution environment has several methods
+to read from files using various methods: you can just read them line by line,
+as CSV files, or using completely custom data input formats. To just read
+a text file as a sequence of lines, you can use:
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+DataSet<String> text = env.readTextFile("file:///path/to/file");
+{% endhighlight %}
+
+This will give you a `DataSet` on which you can then apply transformations. For
+more information on data sources and input formats, please refer to
+[Data Sources](#data_sources).
+
+Once you have a `DataSet` you can apply transformations to create a new
+`DataSet` which you can then write to a file, transform again, or
+combine with other `DataSet`s. You apply transformations by calling
+methods on `DataSet` with your own custom transformation function. For example,
+a map transformation looks like this:
+
+{% highlight java %}
+DataSet<String> input = ...;
+
+DataSet<Integer> tokenized = text.map(new MapFunction<String, Integer>() {
+    @Override
+    public Integer map(String value) {
+        return Integer.parseInt(value);
+    }
+});
+{% endhighlight %}
+
+This will create a new `DataSet` by converting every String in the original
+set to an Integer. For more information and a list of all the transformations,
+please refer to [Transformations](#transformations).
+
+Once you have a `DataSet` that needs to be written to disk you call one
+of these methods on `DataSet`:
+
+{% highlight java %}
+writeAsText(String path)
+writeAsCsv(String path)
+write(FileOutputFormat<T> outputFormat, String filePath)
+
+print()
+{% endhighlight %}
+
+The last method is only useful for developing/debugging on a local machine,
+it will output the contents of the `DataSet` to standard output. (Note that in
+a cluster, the result goes to the standard out stream of the cluster nodes and ends
+up in the *.out* files of the workers).
+The first two do as the name suggests, the third one can be used to specify a
+custom data output format. Keep in mind, that these calls do not actually
+write to a file yet. Only when your program is completely specified and you
+call the `execute` method on your `ExecutionEnvironment` are all the
+transformations executed and is data written to disk. Please refer
+to [Data Sinks](#data_sinks) for more information on writing to files and also
+about custom data output formats.
+
+Once you specified the complete program you need to call `execute` on
+the `ExecutionEnvironment`. This will either execute on your local
+machine or submit your program for execution on a cluster, depending on
+how you created the execution environment.
+
+[Back to top](#top)
+
+
+Lazy Evaluation
+---------------
+
+All Flink programs are executed lazily: When the program's main method is executed, the data loading
+and transformations do not happen directly. Rather, each operation is created and added to the
+program's plan. The operations are actually executed when one of the `execute()` methods is invoked
+on the ExecutionEnvironment object. Whether the program is executed locally or on a cluster depends
+on the environment of the program.
+
+The lazy evaluation lets you construct sophisticated programs that Flink executes as one
+holistically planned unit.
+
+[Back to top](#top)
+
+
+Transformations
+---------------
+
+Data transformations transform one or more DataSets into a new DataSet. Programs can combine
+multiple transformations into sophisticated assemblies.
+
+This section gives a brief overview of the available transformations. The [transformations
+documentation](java_api_transformations.html) has full description of all transformations with
+examples.
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-left" style="width: 20%">Transformation</th>
+      <th class="text-center">Description</th>
+    </tr>
+  </thead>
+
+  <tbody>
+    <tr>
+      <td><strong>Map</strong></td>
+      <td>
+        <p>Takes one element and produces one element.</p>
+{% highlight java %}
+data.map(new MapFunction<String, Integer>() {
+  public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>FlatMap</strong></td>
+      <td>
+        <p>Takes one element and produces zero, one, or more elements. </p>
+{% highlight java %}
+data.flatMap(new FlatMapFunction<String, String>() {
+  public void flatMap(String value, Collector<String> out) {
+    for (String s : value.split(" ")) {
+      out.collect(s);
+    }
+  }
+});
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>MapPartition</strong></td>
+      <td>
+        <p>Transforms a parallel partition in a single function call. The function get the partition
+        as an `Iterable` stream and can produce an arbitrary number of result values. The number of
+        elements in each partition depends on the degree-of-parallelism and previous operations.</p>
+{% highlight java %}
+data.mapPartition(new MapPartitionFunction<String, Long>() {
+  public void mapPartition(Iterable<String> values, Collector<Long> out) {
+    long c = 0;
+    for (String s : values) {
+      c++;
+    }
+    out.collect(c);
+  }
+});
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Filter</strong></td>
+      <td>
+        <p>Evaluates a boolean function for each element and retains those for which the function
+        returns true.</p>
+{% highlight java %}
+data.filter(new FilterFunction<Integer>() {
+  public boolean filter(Integer value) { return value > 1000; }
+});
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Reduce</strong></td>
+      <td>
+        <p>Combines a group of elements into a single element by repeatedly combining two elements
+        into one. Reduce may be applied on a full data set, or on a grouped data set.</p>
+{% highlight java %}
+data.reduce(new ReduceFunction<Integer> {
+  public Integer reduce(Integer a, Integer b) { return a + b; }
+});
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>ReduceGroup</strong></td>
+      <td>
+        <p>Combines a group of elements into one or more elements. ReduceGroup may be applied on a
+        full data set, or on a grouped data set.</p>
+{% highlight java %}
+data.reduceGroup(new GroupReduceFunction<Integer, Integer> {
+  public void reduceGroup(Iterable<Integer> values, Collector<Integer> out) {
+    int prefixSum = 0;
+    for (Integer i : values) {
+      prefixSum += i;
+      out.collect(prefixSum);
+    }
+  }
+});
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Aggregate</strong></td>
+      <td>
+        <p>Aggregates a group of values into a single value. Aggregation functions can be thought of
+        as built-in reduce functions. Aggregate may be applied on a full data set, or on a grouped
+        data set.</p>
+{% highlight java %}
+Dataset<Tuple3<Integer, String, Double>> input = // [...]
+DataSet<Tuple3<Integer, String, Double>> output = input.aggregate(SUM, 0).and(MIN, 2);
+{% endhighlight %}
+	<p>You can also use short-hand syntax for minimum, maximum, and sum aggregations.</p>
+	{% highlight java %}
+	Dataset<Tuple3<Integer, String, Double>> input = // [...]
+DataSet<Tuple3<Integer, String, Double>> output = input.sum(0).andMin(2);
+	{% endhighlight %}
+      </td>
+    </tr>
+
+    </tr>
+      <td><strong>Join</strong></td>
+      <td>
+        Joins two data sets by creating all pairs of elements that are equal on their keys.
+        Optionally uses a JoinFunction to turn the pair of elements into a single element, or a
+        FlatJoinFunction to turn the pair of elements into arbitararily many (including none)
+        elements. See <a href="#defining-keys">keys</a> on how to define join keys.
+{% highlight java %}
+result = input1.join(input2)
+               .where(0)       // key of the first input (tuple field 0)
+               .equalTo(1);    // key of the second input (tuple field 1)
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>CoGroup</strong></td>
+      <td>
+        <p>The two-dimensional variant of the reduce operation. Groups each input on one or more
+        fields and then joins the groups. The transformation function is called per pair of groups.
+        See <a href="#defining-keys">keys</a> on how to define coGroup keys.</p>
+{% highlight java %}
+data1.coGroup(data2)
+     .where(0)
+     .equalTo(1)
+     .with(new CoGroupFunction<String, String, String>() {
+         public void coGroup(Iterable<String> in1, Iterable<String> in2, Collector<String> out) {
+           out.collect(...);
+         }
+      });
+{% endhighlight %}
+      </td>
+    </tr>
+
+    <tr>
+      <td><strong>Cross</strong></td>
+      <td>
+        <p>Builds the Cartesian product (cross product) of two inputs, creating all pairs of
+        elements. Optionally uses a CrossFunction to turn the pair of elements into a single
+        element</p>
+{% highlight java %}
+DataSet<Integer> data1 = // [...]
+DataSet<String> data2 = // [...]
+DataSet<Tuple2<Integer, String>> result = data1.cross(data2);
+{% endhighlight %}
+      </td>
+    </tr>
+    <tr>
+      <td><strong>Union</strong></td>
+      <td>
+        <p>Produces the union of two data sets. This operation happens implicitly if more than one
+        data set is used for a specific function input.</p>
+{% highlight java %}
+DataSet<String> data1 = // [...]
+DataSet<String> data2 = // [...]
+DataSet<String> result = data1.union(data2);
+{% endhighlight %}
+      </td>
+    </tr>
+  </tbody>
+</table>
+
+----------
+
+The following transformations are available on data sets of Tuples:
+
+<table class="table table-bordered">
+  <thead>
+    <tr>
+      <th class="text-left" style="width: 20%">Transformation</th>
+      <th class="text-center">Description</th>
+    </tr>
+  </thead>
+  <tbody>
+   <tr>
+      <td><strong>Project</strong></td>
+      <td>
+        <p>Selects a subset of fields from the tuples</p>
+{% highlight java %}
+DataSet<Tuple3<Integer, Double, String>> in = // [...]
+DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
+{% endhighlight %}
+      </td>
+    </tr>
+  </tbody>
+</table>
+
+The [parallelism](#parallelism) of a transformation can be defined by `setParallelism(int)`.
+`name(String)` assigns a custom name to a transformation which is helpful for debugging. The same is
+possible for [Data Sources](#data_sources) and [Data Sinks](#data_sinks).
+
+[Back to Top](#top)
+
+
+Defining Keys
+-------------
+
+Some transformations (join, coGroup) require that a key is defined on
+its argument DataSets, and other transformations (Reduce, GroupReduce,
+Aggregate) allow that the DataSet is grouped on a key before they are
+applied.
+
+A DataSet is grouped as
+{% highlight java %}
+DataSet<...> input = // [...]
+DataSet<...> reduced = input
+	.groupBy(/*define key here*/)
+	.reduceGroup(/*do something*/);
+{% endhighlight %}
+
+The data model of Flink is not based on key-value pairs. Therefore,
+you do not need to physically pack the data set types into keys and
+values. Keys are "virtual": they are defined as functions over the
+actual data to guide the grouping operator.
+
+The simplest case is grouping a data set of Tuples on one or more
+fields of the Tuple:
+{% highlight java %}
+DataSet<Tuple3<Integer,String,Long>> input = // [...]
+DataSet<Tuple3<Integer,String,Long> grouped = input
+	.groupBy(0)
+	.reduceGroup(/*do something*/);
+{% endhighlight %}
+
+The data set is grouped on the first field of the tuples (the one of
+Integer type). The GroupReduceFunction will thus receive groups with
+the same value of the first field.
+
+{% highlight java %}
+DataSet<Tuple3<Integer,String,Long>> input = // [...]
+DataSet<Tuple3<Integer,String,Long> grouped = input
+	.groupBy(0,1)
+	.reduce(/*do something*/);
+{% endhighlight %}
+
+The data set is grouped on the composite key consisting of the first and the
+second fields, therefore the GroupReduceFuntion will receive groups
+with the same value for both fields.
+
+In general, key definition is done via a "key selector" function, which
+takes as argument one dataset element and returns a key of an
+arbitrary data type by performing an arbitrary computation on this
+element. For example:
+{% highlight java %}
+// some ordinary POJO
+public class WC {public String word; public int count;}
+DataSet<WC> words = // [...]
+DataSet<WC> wordCounts = words
+                         .groupBy(
+                           new KeySelector<WC, String>() {
+                             public String getKey(WC wc) { return wc.word; }
+                           })
+                         .reduce(/*do something*/);
+{% endhighlight %}
+
+Remember that keys are not only used for grouping, but also joining and matching data sets:
+{% highlight java %}
+// some POJO
+public class Rating {
+  public String name;
+  public String category;
+  public int points;
+}
+DataSet<Rating> ratings = // [...]
+DataSet<Tuple2<String, Double>> weights = // [...]
+DataSet<Tuple2<String, Double>>
+            weightedRatings =
+            ratings.join(weights)
+
+                   // key of the first input
+                   .where(new KeySelector<Rating, String>() {
+                            public String getKey(Rating r) { return r.category; }
+                          })
+
+                   // key of the second input
+                   .equalTo(new KeySelector<Tuple2<String, Double>, String>() {
+                              public String getKey(Tuple2<String, Double> t) { return t.f0; }
+                            });
+{% endhighlight %}
+
+[Back to top](#top)
+
+
+Functions
+---------
+
+You can define a user-defined function and pass it to the DataSet
+transformations in several ways:
+
+#### Implementing an interface
+
+The most basic way is to implement one of the provided interfaces:
+
+{% highlight java %}
+class MyMapFunction implements MapFunction<String, Integer> {
+  public Integer map(String value) { return Integer.parseInt(value); }
+});
+data.map (new MyMapFunction());
+{% endhighlight %}
+
+#### Anonymous classes
+
+You can pass a function as an anonmymous class:
+{% highlight java %}
+data.map(new MapFunction<String, Integer> () {
+  public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+
+#### Java 8 Lambdas
+
+***Warning: Lambdas are currently only supported for filter and reduce
+   transformations***
+
+{% highlight java %}
+DataSet<String> data = // [...]
+data.filter(s -> s.startsWith("http://"));
+{% endhighlight %}
+
+{% highlight java %}
+DataSet<Integer> data = // [...]
+data.reduce((i1,i2) -> i1 + i2);
+{% endhighlight %}
+
+#### Rich functions
+
+All transformations that take as argument a user-defined function can
+instead take as argument a *rich* function. For example, instead of
+{% highlight java %}
+class MyMapFunction implements MapFunction<String, Integer> {
+  public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+you can write
+{% highlight java %}
+class MyMapFunction extends RichMapFunction<String, Integer> {
+  public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+and pass the function as usual to a `map` transformation:
+{% highlight java %}
+data.map(new MyMapFunction());
+{% endhighlight %}
+
+Rich functions can also be defined as an anonymous class:
+{% highlight java %}
+data.map (new RichMapFunction<String, Integer>() {
+  public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+
+Rich functions provide, in addition to the user-defined function (map,
+reduce, etc), four methods: `open`, `close`, `getRuntimeContext`, and
+`setRuntimeContext`. These are useful for creating and finalizing
+local state, accessing broadcast variables (see
+[Broadcast Variables](#broadcast_variables), and for accessing runtime
+information such as accumulators and counters (see
+[Accumulators and Counters](#accumulators_counters), and information
+on iterations (see [Iterations](#iterations)).
+
+In particular for the `reduceGroup` transformation, using a rich
+function is the only way to define an optional `combine` function. See
+the
+[transformations documentation](java_api_transformations.html)
+for a complete example.
+
+[Back to top](#top)
+
+
+Data Types
+----------
+
+The Java API is strongly typed: All data sets and transformations accept typed elements. This
+catches type errors very early and supports safe refactoring of programs. The API supports various
+different data types for the input and output of operators. Both `DataSet` and functions like
+`MapFunction`, `ReduceFunction`, etc. are parameterized with data types using Java generics in order
+to ensure type-safety.
+
+There are four different categories of data types, which are treated slightly different:
+
+1. **Regular Types**
+2. **Tuples**
+3. **Values**
+4. **Hadoop Writables**
+
+
+#### Regular Types
+
+Out of the box, the Java API supports all common basic Java types: `Byte`, `Short`, `Integer`,
+`Long`, `Float`, `Double`, `Boolean`, `Character`, `String`.
+
+Furthermore, you can use the vast majority of custom Java classes. Restrictions apply to classes
+containing fields that cannot be serialized, like File pointers, I/O streams, or other native
+resources. Classes that follow the Java Beans conventions work well in general. The following
+defines a simple example class to illustrate how you can use custom classes:
+
+{% highlight java %}
+public class WordWithCount {
+
+    public String word;
+    public int count;
+
+    public WordCount() {}
+
+    public WordCount(String word, int count) {
+        this.word = word;
+        this.count = count;
+    }
+}
+{% endhighlight %}
+
+You can use all of those types to parameterize `DataSet` and function implementations, e.g.
+`DataSet<String>` for a `String` data set or `MapFunction<String, Integer>` for a mapper from
+`String` to `Integer`.
+
+{% highlight java %}
+// using a basic data type
+DataSet<String> numbers = env.fromElements("1", "2");
+
+numbers.map(new MapFunction<String, Integer>() {
+    @Override
+    public String map(String value) throws Exception {
+        return Integer.parseInt(value);
+    }
+});
+
+// using a custom class
+DataSet<WordCount> wordCounts = env.fromElements(
+    new WordCount("hello", 1),
+    new WordCount("world", 2));
+
+wordCounts.map(new MapFunction<WordCount, Integer>() {
+    @Override
+    public String map(WordCount value) throws Exception {
+        return value.count;
+    }
+});
+{% endhighlight %}
+
+When working with operators that require a Key for grouping or matching records
+you need to implement a `KeySelector` for your custom type (see
+[Defining Keys](#defining-keys)).
+
+{% highlight java %}
+wordCounts.groupBy(new KeySelector<WordCount, String>() {
+    public String getKey(WordCount v) {
+        return v.word;
+    }
+}).reduce(new MyReduceFunction());
+{% endhighlight %}
+
+#### Tuples
+
+You can use the `Tuple` classes for composite types. Tuples contain a fix number of fields of
+various types. The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple
+can be an arbitrary Flink type - including further tuples, resulting in nested tuples. Fields of a
+Tuple can be accessed directly using the fields `tuple.f4`, or using the generic getter method
+`tuple.getField(int position)`. The field numbering starts with 0. Note that this stands in contrast
+to the Scala tuples, but it is more consistent with Java's general indexing.
+
+{% highlight java %}
+DataSet<Tuple2<String, Integer>> wordCounts = env.fromElements(
+    new Tuple2<String, Integer>("hello", 1),
+    new Tuple2<String, Integer>("world", 2));
+
+wordCounts.map(new MapFunction<Tuple2<String, Integer>, Integer>() {
+    @Override
+    public String map(Tuple2<String, Integer> value) throws Exception {
+        return value.f1;
+    }
+});
+{% endhighlight %}
+
+When working with operators that require a Key for grouping or matching records,
+Tuples let you simply specify the positions of the fields to be used as key. You can specify more
+than one position to use composite keys (see [Section Data Transformations](#transformations)).
+
+{% highlight java %}
+wordCounts
+    .groupBy(0)
+    .reduce(new MyReduceFunction());
+{% endhighlight %}
+
+In order to access fields more intuitively and to generate more readable code, it is also possible
+to extend a subclass of `Tuple`. You can add getters and setters with custom names that delegate to
+the field positions. See this
+{% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java "example" %} for an
+illustration how to make use of that mechanism.
+
+
+#### Values
+
+*Value types describe their serialization and deserialization manually. Instead of going through a
+*general purpose serialization framework, they provide custom code for those operations by means
+*implementing the `org.apache.flinktypes.Value` interface with the methods `read` and `write`. Using
+*a Value type is reasonable when general purpose serialization would be highly inefficient. An
+*example would be a data type that implements a sparse vector of elements as an array. Knowing that
+*the array is mostly zero, one can use a special encoding for the non-zero elements, while the
+*general purpose serialization would simply write all array elements.
+
+The `org.apache.flinktypes.CopyableValue` interface supports manual internal cloning logic in a
+similar way.
+
+Flink comes with pre-defined Value types that correspond to Java's basic data types. (`ByteValue`,
+`ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`,
+`BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can
+be altered, allowing programmers to reuse objects and take pressure off the garbage collector.
+
+
+#### Hadoop Writables
+
+You can use types that implement the `org.apache.hadoop.Writable` interface. The serialization logic
+defined in the `write()`and `readFields()` methods will be used for serialization.
+
+
+#### Type Erasure & Type Inferrence
+
+The Java compiler throws away much of the generic type information after the compilation. This is
+known as *type erasure* in Java. It means that at runtime, an instance of an object does not know
+its generic type any more. For example, instances of `DataSet<String>` and `DataSet<Long>` look the
+same to the JVM.
+
+Flink requires type information at the time when it prepares the program for execution (when the
+main method of the program is called). The Flink Java API tries to reconstruct the type information
+that was thrown away in various ways and store it explicitly in the data sets and operators. You can
+retrieve the type via `DataSet.getType()`. The method returns an instance of `TypeInformation`,
+which is Flink's internal way of representing types.
+
+The type inference has its limits and needs the "cooperation" of the programmer in some cases.
+Examples for that are methods that create data sets from collections, such as
+`ExecutionEnvironment.fromCollection(),` where you can pass an argument that describes the type. But
+also generic functions like `MapFunction<I, O>` may need extra type information.
+
+The
+{% gh_link /flink-java/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java "ResultTypeQueryable" %}
+interface can be implemented by input formats and functions to tell the API
+explicitly about their return type. The *input types* that the functions are invoked with can
+usually be inferred by the result types of the previous operations.
+
+[Back to top](#top)
+
+
+Data Sources
+------------
+
+Data sources create the initial data sets, such as from files or from Java collections. The general
+mechanism of of creating data sets is abstracted behind an
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java "InputFormat"%}.
+Flink comes
+with several built-in formats to create data sets from common file formats. Many of them have
+shortcut methods on the *ExecutionEnvironment*.
+
+File-based:
+
+- `readTextFile(path)` / `TextInputFormat` - Reads files line wise and returns them as Strings.
+- `readTextFileWithValue(path)` / `TextValueInputFormat` - Reads files line wise and returns them as
+  StringValues. StringValues are mutable strings.
+- `readCsvFile(path)` / `CsvInputFormat` - Parses files of comma (or another char) delimited fields.
+  Returns a DataSet of tuples. Supports the basic java types and their Value counterparts as field
+  types.
+
+Collection-based:
+
+- `fromCollection(Collection)` - Creates a data set from the Java Java.util.Collection. All elements
+  in the collection must be of the same type.
+- `fromCollection(Iterator, Class)` - Creates a data set from an iterator. The class specifies the
+  data type of the elements returned by the iterator.
+- `fromElements(T ...)` - Creates a data set from the given sequence of objects. All objects must be
+  of the same type.
+- `fromParallelCollection(SplittableIterator, Class)` - Creates a data set from an iterator, in
+  parallel. The class specifies the data type of the elements returned by the iterator.
+- `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in
+  parallel.
+
+Generic:
+
+- `createInput(path)` / `InputFormat` - Accepts a generic input format.
+
+**Examples**
+
+{% highlight java %}
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// read text file from local files system
+DataSet<String> localLines = env.readTextFile("file:///path/to/my/textfile");
+
+// read text file from a HDFS running at nnHost:nnPort
+DataSet<String> hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile");
+
+// read a CSV file with three fields
+DataSet<Tuple3<Integer, String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+	                       .types(Integer.class, String.class, Double.class);
+
+// read a CSV file with five fields, taking only two of them
+DataSet<Tuple2<String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+                               .includeFields("10010")  // take the first and the fourth fild
+	                       .types(String.class, Double.class);
+
+// create a set from some given elements
+DataSet<String> value = env.fromElements("Foo", "bar", "foobar", "fubar");
+
+// generate a number sequence
+DataSet<Long> numbers = env.generateSequence(1, 10000000);
+
+// Read data from a relational database using the JDBC input format
+DataSet<Tuple2<String, Integer> dbData = 
+    env.createInput(
+      // create and configure input format
+      JDBCInputFormat.buildJDBCInputFormat()
+                     .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                     .setDBUrl("jdbc:derby:memory:persons")
+                     .setQuery("select name, age from persons")
+                     .finish(),
+      // specify type information for DataSet
+      new TupleTypeInfo(Tuple2.class, STRING_TYPE_INFO, INT_TYPE_INFO)
+    );
+
+// Note: Flink's program compiler needs to infer the data types of the data items which are returned
+// by an InputFormat. If this information cannot be automatically inferred, it is necessary to
+// manually provide the type information as shown in the examples above. {% endhighlight %}
+
+[Back to top](#top)
+
+Data Sinks
+----------
+
+Data sinks consume DataSets and are used to store or return them. Data sink operations are described
+using an
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java "OutputFormat" %}.
+Flink comes with a variety of built-in output formats that are encapsulated behind operations on the
+DataSet:
+
+- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are
+  obtained by calling the *toString()* method of each element.
+- `writeAsFormattedText()` / `TextOutputFormat` - Write elements line-wise as Strings. The Strings
+  are obtained by calling a user-defined *format()* method for each element.
+- `writeAsCsv` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field
+  delimiters are configurable. The value for each field comes from the *toString()* method of the objects.
+- `print()` / `printToErr()` - Prints the *toString()* value of each element on the
+  standard out / strandard error stream.
+- `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports
+  custom object-to-bytes conversion.
+- `output()`/ `OutputFormat` - Most generic output method, for data sinks that are not file based
+  (such as storing the result in a database).
+
+A DataSet can be input to multiple operations. Programs can write or print a data set and at the
+same time run additional transformations on them.
+
+**Examples**
+
+Standard data sink methods:
+
+{% highlight java %}
+// text data 
+DataSet<String> textData = // [...]
+
+// write DataSet to a file on the local file system
+textData.writeAsText("file:///my/result/on/localFS");
+
+// write DataSet to a file on a HDFS with a namenode running at nnHost:nnPort
+textData.writeAsText("hdfs://nnHost:nnPort/my/result/on/localFS");
+
+// write DataSet to a file and overwrite the file if it exists
+textData.writeAsText("file:///my/result/on/localFS", WriteMode.OVERWRITE);
+
+// tuples as lines with pipe as the separator "a|b|c"
+DataSet<Tuple3<String, Integer, Double>> values = // [...]
+values.writeAsCsv("file:///path/to/the/result/file", "\n", "|");
+
+// this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines
+values.writeAsText("file:///path/to/the/result/file");
+
+// this wites values as strings using a user-defined TextFormatter object
+values.writeAsFormattedText("file:///path/to/the/result/file",
+    new TextFormatter<Tuple2<Integer, Integer>>() {
+        public String format (Tuple2<Integer, Integer> value) {
+            return value.f1 + " - " + value.f0;
+        }
+    });
+{% endhighlight %}
+
+Using a custom output format:
+
+{% highlight java %}
+DataSet<Tuple3<String, Integer, Double>> myResult = [...]
+
+// write Tuple DataSet to a relational database
+myResult.output(
+    // build and configure OutputFormat
+    JDBCOutputFormat.buildJDBCOutputFormat()
+                    .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                    .setDBUrl("jdbc:derby:memory:persons")
+                    .setQuery("insert into persons (name, age, height) values (?,?,?)")
+                    .finish()
+    );
+{% endhighlight %}
+
+[Back to top](#top)
+
+
+Debugging
+---------
+
+Before running a data analysis program on a large data set in a distributed cluster, it is a good
+idea to make sure that the implemented algorithm works as desired. Hence, implementing data analysis
+programs is usually an incremental process of checking results, debugging, and improving.
+
+Flink provides a few nice features to significantly ease the development process of data analysis
+programs by supporting local debugging from within an IDE, injection of test data, and collection of
+result data. This section give some hints how to ease the development of Flink programs.
+
+### Local Execution Environment
+
+A `LocalEnvironment` starts a Flink system within the same JVM process it was created in. If you
+start the LocalEnvironement from an IDE, you can set breakpoint in your code and easily debug your
+program.
+
+A LocalEnvironment is created and used as follows:
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+DataSet<String> lines = env.readTextFile(pathToTextFile);
+// build your program
+
+env.execute();
+
+{% endhighlight %}
+
+### Collection Data Sources and Sinks
+
+Providing input for an analysis program and checking its output is cumbersome done by creating input
+files and reading output files. Flink features special data sources and sinks which are backed by
+Java collections to ease testing. Once a program has been tested, the sources and sinks can be
+easily replaced by sources and sinks that read from / write to external data stores such as HDFS.
+
+Collection data sources can be used as follows:
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+// Create a DataSet from a list of elements
+DataSet<Integer> myInts = env.fromElements(1, 2, 3, 4, 5);
+
+// Create a DataSet from any Java collection
+List<Tuple2<String, Integer>> data = ...
+DataSet<Tuple2<String, Integer>> myTuples = env.fromCollection(data);
+
+// Create a DataSet from an Iterator
+Iterator<Long> longIt = ...
+DataSet<Long> myLongs = env.fromCollection(longIt, Long.class);
+{% endhighlight %}
+
+**Note: Currently, the collection data source requires that data types and iterators implement
+**`Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of
+**parallelism = 1).
+
+A collection data sink is specified as follows:
+
+{% highlight java %}
+DataSet<Tuple2<String, Integer>> myResult = ...
+
+List<Tuple2<String, Integer>> outData = new ArrayList<Tuple2<String, Integer>>();
+myResult.output(new LocalCollectionOutputFormat(outData));
+{% endhighlight %}
+
+**Note:** Currently, the collection data sink is restricted to local execution, as a debugging tool.
+
+[Back to top](#top)
+
+
+Iteration Operators
+-------------------
+
+Iterations implement loops in Flink programs. The iteration operators encapsulate a part of the
+program and execute it repeatedly, feeding back the result of one iteration (the partial solution)
+into the next iteration. There are two types of iterations in Flink: **BulkIteration** and
+**DeltaIteration**.
+
+This section provides quick examples on how to use both operators. Check out the [Introduction to
+Iterations](iterations.html) page for a more detailed introduction.
+
+#### Bulk Iterations
+
+To create a BulkIteration call the `iterate(int)` method of the `DataSet` the iteration should start
+at. This will return an `IterativeDataSet`, which can be transformed with the regular operators. The
+single argument to the iterate call specifies the maximum number of iterations.
+
+To specify the end of an iteration call the `closeWith(DataSet)` method on the `IterativeDataSet` to
+specify which transformation should be fed back to the next iteration. You can optionally specify a
+termination criterion with `closeWith(DataSet, DataSet)`, which evaluates the second DataSet and
+terminates the iteration, if this DataSet is empty. If no termination criterion is specified, the
+iteration terminates after the given maximum number iterations.
+
+The following example iteratively estimates the number Pi. The goal is to count the number of random
+points, which fall into the unit circle. In each iteration, a random point is picked. If this point
+lies inside the unit circle, we increment the count. Pi is then estimated as the resulting count
+divided by the number of iterations multiplied by 4.
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// Create initial IterativeDataSet
+IterativeDataSet<Integer> initial = env.fromElements(0).iterate(10000);
+
+DataSet<Integer> iteration = initial.map(new MapFunction<Integer, Integer>() {
+    @Override
+    public Integer map(Integer i) throws Exception {
+        double x = Math.random();
+        double y = Math.random();
+
+        return i + ((x * x + y * y < 1) ? 1 : 0);
+    }
+});
+
+// Iteratively transform the IterativeDataSet
+DataSet<Integer> count = initial.closeWith(iteration);
+
+count.map(new MapFunction<Integer, Double>() {
+    @Override
+    public Double map(Integer count) throws Exception {
+        return count / (double) 10000 * 4;
+    }
+}).print();
+
+env.execute("Iterative Pi Example");
+{% endhighlight %}
+
+You can also check out the
+{% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java "K-Means example" %},
+which uses a BulkIteration to cluster a set of unlabeled points.
+
+#### Delta Iterations
+
+Delta iterations exploit the fact that certain algorithms do not change every data point of the
+solution in each iteration.
+
+In addition to the partial solution that is fed back (called workset) in every iteration, delta
+iterations maintain state across iterations (called solution set), which can be updated through
+deltas. The result of the iterative computation is the state after the last iteration. Please refer
+to the [Introduction to Iterations](iterations.html) for an overview of the basic principle of delta
+iterations.
+
+Defining a DeltaIteration is similar to defining a BulkIteration. For delta iterations, two data
+sets form the input to each iteration (workset and solution set), and two data sets are produced as
+the result (new workset, solution set delta) in each iteration.
+
+To create a DeltaIteration call the `iterateDelta(DataSet, int, int)` (or `iterateDelta(DataSet,
+int, int[])` respectively). This method is called on the initial solution set. The arguments are the
+initial delta set, the maximum number of iterations and the key positions. The returned
+`DeltaIteration` object gives you access to the DataSets representing the workset and solution set
+via the methods `iteration.getWorket()` and `iteration.getSolutionSet()`.
+
+Below is an example for the syntax of a delta iteration
+
+{% highlight java %}
+// read the initial data sets
+DataSet<Tuple2<Long, Double>> initialSolutionSet = // [...]
+
+DataSet<Tuple2<Long, Double>> initialDeltaSet = // [...]
+
+int maxIterations = 100;
+int keyPosition = 0;
+
+DeltaIteration<Tuple2<Long, Double>, Tuple2<Long, Double>> iteration = initialSolutionSet
+    .iterateDelta(initialDeltaSet, maxIterations, keyPosition);
+
+DataSet<Tuple2<Long, Double>> candidateUpdates = iteration.getWorkset()
+    .groupBy(1)
+    .reduceGroup(new ComputeCandidateChanges());
+
+DataSet<Tuple2<Long, Double>> deltas = candidateUpdates
+    .join(iteration.getSolutionSet())
+    .where(0)
+    .equalTo(0)
+    .with(new CompareChangesToCurrent());
+
+DataSet<Tuple2<Long, Double>> nextWorkset = deltas
+    .filter(new FilterByThreshold());
+
+iteration.closeWith(deltas, nextWorkset)
+	.writeAsCsv(outputPath);
+{% endhighlight %}
+
+[Back to top](#top)
+
+
+Semantic Annotations
+-----------
+
+Semantic Annotations give hints about the behavior of a function by telling the system which fields
+in the input are accessed and which are constant between input and output data of a function (copied
+but not modified). Semantic annotations are a powerful means to speed up execution, because they
+allow the system to reason about reusing sort orders or partitions across multiple operations. Using
+semantic annotations may eventually save the program from unnecessary data shuffling or unnecessary
+sorts.
+
+Semantic annotations can be attached to functions through Java annotations, or passed as arguments
+when invoking a function on a DataSet. The following example illustrates that:
+
+{% highlight java %}
+@ConstantFields("1")
+public class DivideFirstbyTwo implements MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
+  @Override
+  public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) {
+    value.f0 /= 2;
+    return value;
+  }
+}
+{% endhighlight %}
+
+The following annotations are currently available:
+
+* `@ConstantFields`: Declares constant fields (forwarded/copied) for functions with a single input
+  data set (Map, Reduce, Filter, ...).
+
+* `@ConstantFieldsFirst`: Declares constant fields (forwarded/copied) for functions with a two input
+  data sets (Join, CoGroup, ...), with respect to the first input data set.
+
+* `@ConstantFieldsSecond`: Declares constant fields (forwarded/copied) for functions with a two
+  input data sets (Join, CoGroup, ...), with respect to the first second data set.
+
+* `@ConstantFieldsExcept`: Declares that all fields are constant, except for the specified fields.
+  Applicable to functions with a single input data set.
+
+* `@ConstantFieldsFirstExcept`: Declares that all fields of the first input are constant, except for
+  the specified fields. Applicable to functions with a two input data sets.
+
+* `@ConstantFieldsSecondExcept`: Declares that all fields of the second input are constant, except
+  for the specified fields. Applicable to functions with a two input data sets.
+
+*(Note: The system currently evaluated annotations only on Tuple data types. 
+his will be extended in the next versions)*
+
+**Note**: It is important to be conservative when providing annotations. Only annotate fields,
+when they are always constant for every call to the function. Otherwise the system has incorrect
+assumptions about the execution and the execution may produce wrong results. If the behavior of the
+operator is not clearly predictable, no annotation should be provided.
+
+[Back to top](#top)
+
+
+Broadcast Variables
+-------------------
+
+Broadcast variables allow you to make a data set available to all parallel instances of an
+operation, in addition to the regular input of the operation. This is useful for auxiliary data
+sets, or data-dependent parameterization. The data set will then be accessible at the operator as an
+`Collection<T>`.
+
+- **Broadcast**: broadcast sets are registered by name via `withBroadcastSet(DataSet, String)`, and
+- **Access**: accessible via `getRuntimeContext().getBroadcastVariable(String)` at the target operator.
+
+{% highlight java %}
+// 1. The DataSet to be broadcasted
+DataSet<Integer> toBroadcast = env.fromElements(1, 2, 3);
+
+DataSet<String> data = env.fromElements("a", "b");
+
+data.map(new MapFunction<String, String>() {
+    @Override
+    public void open(Configuration parameters) throws Exception {
+      // 3. Access the broadcasted DataSet as a Collection
+      Collection<Integer> broadcastSet = getRuntimeContext().getBroadcastVariable("broadcastSetName");
+    }
+
+
+    @Override
+    public String map(String value) throws Exception {
+        ...
+    }
+}).withBroadcastSet(toBroadcast, "broadcastSetName"); // 2. Broadcast the DataSet
+{% endhighlight %}
+
+Make sure that the names (`broadcastSetName` in the previous example) match when registering and
+accessing broadcasted data sets. For a complete example program, have a look at
+{% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java#L96 "KMeans Algorithm" %}.
+
+**Note**: As the content of broadcast variables is kept in-memory on each node, it should not become
+too large. For simpler things like scalar values you can simply make parameters part of the closure
+of a function, or use the `withParameters(...)` method to pass in a configuration.
+
+[Back to top](#top)
+
+
+Program Packaging & Distributed Execution
+-----------------------------------------
+
+As described in the [program skeleton](#skeleton) section, Flink programs can be executed on
+clusters by using the `RemoteEnvironment`. Alternatively, programs can be packaged into JAR Files
+(Java Archives) for execution. Packaging the program is a prerequisite to executing them through the
+[command line interface](cli.html) or the [web interface](web_client.html).
+
+#### Packaging Programs
+
+To support execution from a packaged JAR file via the command line or web interface, a program must
+use the environment obtained by `ExecutionEnvironment.getExecutionEnvironment()`. This environment
+will act as the cluster's environment when the JAR is submitted to the command line or web
+interface. If the Flink program is invoked differently than through these interfaces, the
+environment will act like a local environment.
+
+To package the program, simply export all involved classes as a JAR file. The JAR file's manifest
+must point to the class that contains the program's *entry point* (the class with the `public void
+main(String[])` method). The simplest way to do this is by putting the *main-class* entry into the
+manifest (such as `main-class: org.apache.flinkexample.MyProgram`). The *main-class* attribute is
+the same one that is used by the Java Virtual Machine to find the main method when executing a JAR
+files through the command `java -jar pathToTheJarFile`. Most IDEs offer to include that attribute
+automatically when exporting JAR files.
+
+
+#### Packaging Programs through Plans
+
+Additionally, the Java API supports packaging programs as *Plans*. This method resembles the way
+that the *Scala API* packages programs. Instead of defining a progam in the main method and calling
+`execute()` on the environment, plan packaging returns the *Program Plan*, which is a description of
+the program's data flow. To do that, the program must implement the
+`org.apache.flinkapi.common.Program` interface, defining the `getPlan(String...)` method. The
+strings passed to that method are the command line arguments. The program's plan can be created from
+the environment via the `ExecutionEnvironment#createProgramPlan()` method. When packaging the
+program's plan, the JAR manifest must point to the class implementing the
+`org.apache.flinkapi.common.Program` interface, instead of the class with the main method.
+
+
+#### Summary
+
+The overall procedure to invoke a packaged program is as follows:
+
+1. The JAR's manifest is searched for a *main-class* or *program-class* attribute. If both
+attributes are found, the *program-class* attribute takes precedence over the *main-class*
+attribute. Both the command line and the web interface support a parameter to pass the entry point
+class name manually for cases where the JAR manifest contains neither attribute.
+
+2. If the entry point class implements the `org.apache.flinkapi.common.Program`, then the system
+calls the `getPlan(String...)` method to obtain the program plan to execute. The
+`getPlan(String...)` method was the only possible way of defining a program in the *Record API*
+(see [0.4 docs](http://stratosphere.eu/docs/0.4/)) and is also supported in the new Java API.
+
+3. If the entry point class does not implement the `org.apache.flinkapi.common.Program` interface,
+the system will invoke the main method of the class.
+
+[Back to top](#top)
+
+Accumulators & Counters
+---------------------------
+
+Accumulators are simple constructs with an **add operation** and a **final accumulated result**,
+which is available after the job ended.
+
+The most straightforward accumulator is a **counter**: You can increment it using the
+```Accumulator.add(V value)``` method. At the end of the job Flink will sum up (merge) all partial
+results and send the result to the client. Since accumulators are very easy to use, they can be
+useful during debugging or if you quickly want to find out more about your data.
+
+Flink currently has the following **built-in accumulators**. Each of them implements the
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
+interface.
+
+- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java "__IntCounter__" %},
+  {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java "__LongCounter__" %}
+  and {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java "__DoubleCounter__" %}: 
+  See below for an example using a counter.
+- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java "__Histogram__" %}:
+  A histogram implementation for a discrete number of bins. Internally it is just a map from Integer
+  to Integer. You can use this to compute distributions of values, e.g. the distribution of
+  words-per-line for a word count program.
+
+__How to use accumulators:__
+
+First you have to create an accumulator object (here a counter) in the operator function where you
+want to use it. Operator function here refers to the (anonymous inner) class implementing the user
+defined code for an operator.
+
+{% highlight java %}
+private IntCounter numLines = new IntCounter();
+{% endhighlight %}
+
+Second you have to register the accumulator object, typically in the ```open()``` method of the
+operator function. Here you also define the name.
+
+{% highlight java %}
+getRuntimeContext().addAccumulator("num-lines", this.numLines);
+{% endhighlight %}
+
+You can now use the accumulator anywhere in the operator function, including in the ```open()``` and
+```close()``` methods.
+
+{% highlight java %}
+this.numLines.add(1);
+{% endhighlight %}
+
+The overall result will be stored in the ```JobExecutionResult``` object which is returned when
+running a job using the Java API (currently this only works if the execution waits for the
+completion of the job).
+
+{% highlight java %}
+myJobExecutionResult.getAccumulatorResult("num-lines")
+{% endhighlight %}
+
+All accumulators share a single namespace per job. Thus you can use the same accumulator in
+different operator functions of your job. Flink will internally merge all accumulators with the same
+name.
+
+A note on accumulators and iterations: Currently the result of accumulators is only available after
+the overall job ended. We plan to also make the result of the previous iteration available in the
+next iteration. You can use
+{% gh_link /flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java#L98 "Aggregators" %}
+to compute per-iteration statistics and base the termination of iterations on such statistics.
+
+__Custom accumulators:__
+
+To implement your own accumulator you simply have to write your implementation of the Accumulator
+interface. Feel free to create a pull request if you think your custom accumulator should be shipped
+with Flink.
+
+You have the choice to implement either
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
+or {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java "SimpleAccumulator" %}.
+
+```Accumulator<V,R>``` is most flexible: It defines a type ```V``` for the value to add, and a
+result type ```R``` for the final result. E.g. for a histogram, ```V``` is a number and ```R``` i
+ a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.
+
+[Back to top](#top)
+
+Parallel Execution
+------------------
+
+This section describes how the parallel execution of programs can be configured in Flink. A Flink
+program consists of multiple tasks (operators, data sources, and sinks). A task is split into
+several parallel instances for execution and each parallel instance processes a subset of the task's
+input data. The number of parallel instances of a task is called its *parallelism* or *degree of
+parallelism (DOP)*.
+
+The degree of parallelism of a task can be specified in Flink on different levels.
+
+### Operator Level
+
+The parallelism of an individual operator, data source, or data sink can be defined by calling its
+`setParallelism()` method.  For example, the degree of parallelism of the `Sum` operator in the
+[WordCount](#example) example program can be set to `5` as follows :
+
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+DataSet<String> text = [...]
+DataSet<Tuple2<String, Integer>> wordCounts = text
+    .flatMap(new LineSplitter())
+    .groupBy(0)
+    .sum(1).setParallelism(5);
+wordCounts.print();
+
+env.execute("Word Count Example");
+{% endhighlight %}
+
+### Execution Environment Level
+
+Flink programs are executed in the context of an [execution environmentt](#program-skeleton). An
+execution environment defines a default parallelism for all operators, data sources, and data sinks
+it executes. Execution environment parallelism can be overwritten by explicitly configuring the
+parallelism of an operator.
+
+The default parallelism of an execution environment can be specified by calling the
+`setDefaultLocalParallelism()` method. To execute all operators, data sources, and data sinks of the
+[WordCount](#example) example program with a parallelism of `3`, set the default parallelism of the
+execution environment as follows:
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+env.setDegreeOfParallelism(3);
+
+DataSet<String> text = [...]
+DataSet<Tuple2<String, Integer>> wordCounts = [...]
+wordCounts.print();
+
+env.execute("Word Count Example");
+{% endhighlight %}
+
+### System Level
+
+A system-wide default parallelism for all execution environments can be defined by setting the
+`parallelization.degree.default` property in `./conf/flink-conf.yaml`. See the
+[Configuration]({{site.baseurl}}/config.html) documentation for details.
+
+[Back to top](#top)
+
+Execution Plans
+---------------
+
+Depending on various parameters such as data size or number of machines in the cluster, Flink's
+optimizer automatically chooses an execution strategy for your program. In many cases, it can be
+useful to know how exactly Flink will execute your program.
+
+__Plan Visualization Tool__
+
+Flink 0.5 comes packaged with a visualization tool for execution plans. The HTML document containing
+the visualizer is located under ```tools/planVisualizer.html```. It takes a JSON representation of
+the job execution plan and visualizes it as a graph with complete annotations of execution
+strategies.
+
+The following code shows how to print the execution plan JSON from your program:
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+...
+
+System.out.println(env.getExecutionPlan());
+{% endhighlight %}
+
+
+To visualize the execution plan, do the following:
+
+1. **Open** ```planVisualizer.html``` with your web browser,
+2. **Paste** the JSON string into the text field, and
+3. **Press** the draw button.
+
+After these steps, a detailed execution plan will be visualized.
+
+<img alt="A flink job execution graph." src="img/plan_visualizer2.png" width="80%">
+
+
+__Web Interface__
+
+Flink offers a web interface for submitting and executing jobs. If you choose to use this interface to submit your packaged program, you have the option to also see the plan visualization.
+
+The script to start the webinterface is located under ```bin/start-webclient.sh```. After starting the webclient (per default on **port 8080**), your program can be uploaded and will be added to the list of available programs on the left side of the interface.
+
+You are able to specify program arguments in the textbox at the bottom of the page. Checking the plan visualization checkbox shows the execution plan before executing the actual program.
+
+[Back to top](#top)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/run_example_quickstart.md b/docs/run_example_quickstart.md
index e033c0b..88d6137 100644
--- a/docs/run_example_quickstart.md
+++ b/docs/run_example_quickstart.md
@@ -2,40 +2,46 @@
 title: "Quick Start: Run K-Means Example"
 ---
 
+* This will be replaced by the TOC
+{:toc}
 
 This guide will demonstrate Flink's features by example. You will see how you can leverage Flink's Iteration-feature to find clusters in a dataset using [K-Means clustering](http://en.wikipedia.org/wiki/K-means_clustering). 
 On the way, you will see the compiler, the status interface and the result of the algorithm.
 
 
-#  Generate Input Data
+##  Generate Input Data
 Flink contains a data generator for K-Means.
 
-	# Download Flink
-	wget {{ site.FLINK_DOWNLOAD_URL_HADOOP_1_STABLE }}
-	tar xzf flink-*.tgz 
-	cd flink-*
-	mkdir kmeans
-	cd kmeans
-	# Run data generator
-	java -cp  ../examples/flink-java-examples-{{ site.FLINK_VERSION_STABLE }}-KMeans.jar org.apache.flink.example.java.clustering.util.KMeansDataGenerator 500 10 0.08
-	cp /tmp/points .
-	cp /tmp/centers .
-
+~~~bash
+# Download Flink
+wget {{ site.FLINK_DOWNLOAD_URL_HADOOP_1_STABLE }}
+tar xzf flink-*.tgz 
+cd flink-*
+mkdir kmeans
+cd kmeans
+# Run data generator
+java -cp  ../examples/flink-java-examples-{{ site.FLINK_VERSION_STABLE }}-KMeans.jar org.apache.flink.example.java.clustering.util.KMeansDataGenerator 500 10 0.08
+cp /tmp/points .
+cp /tmp/centers .
+~~~
 
 The generator has the following arguments:
 
-	KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]
+~~~bash
+KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]
+~~~
 
 The _relative standard deviation_ is an interesting tuning parameter: it determines the closeness of the points to the centers.
 
 The `kmeans/` directory should now contain two files: `centers` and `points`.
 
 
-# Review Input Data
+## Review Input Data
 Use the `plotPoints.py` tool to review the result of the data generator. [Download Python Script](quickstart/plotPoints.py)
-```bash
+
+~~~ bash
 python plotPoints.py points points input
-```
+~~~ 
 
 
 Note: You might have to install [matplotlib](http://matplotlib.org/) (`python-matplotlib` package on Ubuntu) to use the Python script.
@@ -49,7 +55,7 @@ The following overview presents the impact of the different standard deviations
 |<img src="img/quickstart-example/kmeans003.png" alt="example1" style="width: 275px;"/>|<img src="img/quickstart-example/kmeans008.png" alt="example2" style="width: 275px;"/>|<img src="img/quickstart-example/kmeans015.png" alt="example3" style="width: 275px;"/>|
 
 
-# Run Clustering
+## Run Clustering
 We are using the generated input data to run the clustering using a Flink job.
 
 	# go to the Flink-root directory
@@ -59,7 +65,7 @@ We are using the generated input data to run the clustering using a Flink job.
 	# Start Flink web client
 	./bin/start-webclient.sh
 
-# Review Flink Compiler
+## Review Flink Compiler
 The Flink webclient allows to submit Flink programs using a graphical user interface.
 
 <div class="row" style="padding-top:15px">
@@ -107,13 +113,13 @@ The Flink webclient allows to submit Flink programs using a graphical user inter
 </div>
 
 
-# Analyze the Result
+## Analyze the Result
 
 Use the [Python Script](quickstart/plotPoints.py) again to visualize the result
 
-```bash
+~~~bash
 python plotPoints.py result result result-pdf
-```
+~~~
 
 The following three pictures show the results for the sample input above. Play around with the parameters (number of iterations, number of clusters) to see how they affect the result.
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/scala_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_quickstart.md b/docs/scala_api_quickstart.md
index c376669..ea3e9ae 100644
--- a/docs/scala_api_quickstart.md
+++ b/docs/scala_api_quickstart.md
@@ -2,13 +2,16 @@
 title: "Quickstart: Scala API"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Start working on your Flink Scala program in a few simple steps.
 
-#Requirements
+## Requirements
 The only requirements are working __Maven 3.0.4__ (or higher) and __Java 6.x__ (or higher) installations.
 
 
-#Create Project
+## Create Project
 Use one of the following commands to __create a project__:
 
 <ul class="nav nav-tabs" style="border-bottom: none;">
@@ -33,7 +36,7 @@ $ mvn archetype:generate                             \
 </div>
 
 
-#Inspect Project
+## Inspect Project
 There will be a __new directory in your working directory__. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId.
 
 The sample project is a __Maven project__, which contains a sample scala _job_ that implements Word Count. Please note that the _RunJobLocal_ and _RunJobRemote_ objects allow you to start Flink in a development/testing mode.</p>
@@ -52,11 +55,11 @@ We recommend to __import this project into your IDE__. For Eclipse, you need the
 The IntelliJ IDE also supports Maven and offers a plugin for Scala development.
 
 
-# Build Project
+## Build Project
 
 If you want to __build your project__, go to your project directory and issue the`mvn clean package` command. You will __find a jar__ that runs on every Flink cluster in __target/flink-project-0.1-SNAPSHOT.jar__.
 
-#Next Steps
+## Next Steps
 
 __Write your application!__
 If you have any trouble, ask on our [Jira page](https://issues.apache.org/jira/browse/FLINK) (open an issue) or on our Mailing list. We are happy to provide help.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/setup_quickstart.md b/docs/setup_quickstart.md
index aa5ac23..c2a30cc 100644
--- a/docs/setup_quickstart.md
+++ b/docs/setup_quickstart.md
@@ -2,30 +2,34 @@
 title: "Quickstart: Setup"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Get Flink up and running in a few simple steps.
 
-# Requirements
+## Requirements
 Flink runs on all __UNIX-like__ environments: __Linux__, __Mac OS X__, __Cygwin__. The only requirement is to have a working __Java 6.x__ (or higher) installation.
 
-# Download
+## Download
 Download the ready to run binary package. Choose the Flink distribution that __matches your Hadoop version__. If you are unsure which version to choose or you just want to run locally, pick the package for Hadoop 1.2.
 
 <ul class="nav nav-tabs">
-   <li class="active"><a href="#bin-hadoop1" data-toggle="tab">Hadoop 1.2</a></li>
-   <li><a href="#bin-hadoop2" data-toggle="tab">Hadoop 2 (YARN)</a></li>
- </ul>
- <div class="tab-content text-center">
-   <div class="tab-pane active" id="bin-hadoop1">
-     <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-1',this.href]);" href="{{site.FLINK_DOWNLOAD_URL_HADOOP_1_STABLE}}"><i class="icon-download"> </i> Download Flink for Hadoop 1.2</a>
-   </div>
-   <div class="tab-pane" id="bin-hadoop2">
-     <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-2',this.href]);" href="{{site.FLINK_DOWNLOAD_URL_HADOOP_2_STABLE}}"><i class="icon-download"> </i> Download Flink for Hadoop 2</a>
-   </div>
- </div>
+  <li class="active"><a href="#bin-hadoop1" data-toggle="tab">Hadoop 1.2</a></li>
+  <li><a href="#bin-hadoop2" data-toggle="tab">Hadoop 2 (YARN)</a></li>
+</ul>
+<p>
+<div class="tab-content text-center">
+  <div class="tab-pane active" id="bin-hadoop1">
+    <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-1',this.href]);" href="{{site.FLINK_DOWNLOAD_URL_HADOOP_1_STABLE}}"><i class="icon-download"> </i> Download Flink for Hadoop 1.2</a>
+  </div>
+  <div class="tab-pane" id="bin-hadoop2">
+    <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-2',this.href]);" href="{{site.FLINK_DOWNLOAD_URL_HADOOP_2_STABLE}}"><i class="icon-download"> </i> Download Flink for Hadoop 2</a>
+  </div>
+</div>
 </p>
 
 
-# Start
+## Start
 You are almost done.
   
 1. Go to the download directory.
@@ -33,46 +37,59 @@ You are almost done.
 3. Start Flink.
 
 
-```bash
+~~~bash
 $ cd ~/Downloads              # Go to download directory
 $ tar xzf flink-*.tgz  # Unpack the downloaded archive
 $ cd flink
 $ bin/start-local.sh          # Start Flink
-```
+~~~
 
-Check the __JobManager's web frontend__ at [http://localhost:8081](http://localhost:8081) and make sure everything is up and running.
+Check the __JobManager's web frontend__ at [http://localhost:8081](http://localhost:8081) and make
+sure everything is up and running.
 
-# Run Example
+## Run Example
 
 Run the __Word Count example__ to see Flink at work.
 
 * __Download test data__:
-```bash
-$ wget -O hamlet.txt http://www.gutenberg.org/cache/epub/1787/pg1787.txt
-```
+
+  ~~~bash
+  $ wget -O hamlet.txt http://www.gutenberg.org/cache/epub/1787/pg1787.txt
+  ~~~ 
+
 * You now have a text file called _hamlet.txt_ in your working directory.
 * __Start the example program__:
-```bash
-$ bin/flink run \
+  
+  ~~~bash
+  $ bin/flink run \
     --jarfile ./examples/flink-java-examples-{{site.FLINK_VERSION_STABLE}}-WordCount.jar \
 
     --arguments file://`pwd`/hamlet.txt file://`pwd`/wordcount-result.txt
-```
+  ~~~
+
 * You will find a file called __wordcount-result.txt__ in your current directory.
   
 
-# Cluster Setup
+## Cluster Setup
   
-__Running Flink on a cluster__ is as easy as running it locally. Having __passwordless SSH__ and __the same directory structure__ on all your cluster nodes lets you use our scripts to control everything.
-
-1. Copy the unpacked __flink__ directory from the downloaded archive to the same file system path on each node of your setup.
-2. Choose a __master node__ (JobManager) and set the `jobmanager.rpc.address` key in `conf/flink-conf.yaml` to its IP or hostname. Make sure that all nodes in your cluster have the same `jobmanager.rpc.address` configured.
-3. Add the IPs or hostnames (one per line) of all __worker nodes__ (TaskManager) to the slaves files in `conf/slaves`.
+__Running Flink on a cluster__ is as easy as running it locally. Having __passwordless SSH__ and
+__the same directory structure__ on all your cluster nodes lets you use our scripts to control
+everything.
+
+1. Copy the unpacked __flink__ directory from the downloaded archive to the same file system path
+on each node of your setup.
+2. Choose a __master node__ (JobManager) and set the `jobmanager.rpc.address` key in
+`conf/flink-conf.yaml` to its IP or hostname. Make sure that all nodes in your cluster have the same
+`jobmanager.rpc.address` configured.
+3. Add the IPs or hostnames (one per line) of all __worker nodes__ (TaskManager) to the slaves files
+in `conf/slaves`.
 
 You can now __start the cluster__ at your master node with `bin/start-cluster.sh`.
 
 
-The following __example__ illustrates the setup with three nodes (with IP addresses from _10.0.0.1_ to _10.0.0.3_ and hostnames _master_, _worker1_, _worker2_) and shows the contents of the configuration files, which need to be accessible at the same path on all machines:
+The following __example__ illustrates the setup with three nodes (with IP addresses from _10.0.0.1_
+to _10.0.0.3_ and hostnames _master_, _worker1_, _worker2_) and shows the contents of the
+configuration files, which need to be accessible at the same path on all machines:
 
 <div class="row">
   <div class="col-md-6 text-center">
@@ -89,14 +106,14 @@ The following __example__ illustrates the setup with three nodes (with IP addres
   <p class="lead text-center">
     /path/to/<strong>flink/<br>conf/slaves</strong>
   <pre>
-    10.0.0.2
-    10.0.0.3</pre>
+10.0.0.2
+10.0.0.3</pre>
   </p>
 </div>
 </div>
 </div>
 
-# Flink on YARN
+## Flink on YARN
 You can easily deploy Flink on your existing __YARN cluster__. 
 
 1. Download the __Flink YARN package__ with the YARN client: [Flink for YARN]({{site.FLINK_DOWNLOAD_URL_YARN_STABLE}})

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/spargel_guide.md
----------------------------------------------------------------------
diff --git a/docs/spargel_guide.md b/docs/spargel_guide.md
index 9d1a5c9..6fe2585 100644
--- a/docs/spargel_guide.md
+++ b/docs/spargel_guide.md
@@ -2,8 +2,8 @@
 title: "Spargel Graph Processing API"
 ---
 
-Spargel
-=======
+* This will be replaced by the TOC
+{:toc}
 
 Spargel is our [Giraph](http://giraph.apache.org) like **graph processing** Java API. It supports basic graph computations, which are run as a sequence of [supersteps](iterations.html#supersteps). Spargel and Giraph both implement the [Bulk Synchronous Parallel (BSP)](https://en.wikipedia.org/wiki/Bulk_Synchronous_Parallel) programming model, propsed by Google's [Pregel](http://googleresearch.blogspot.de/2009/06/large-scale-graph-computing-at-google.html).
 
@@ -21,13 +21,13 @@ The Spargel API is part of the *addons* Maven project. All relevant classes are
 
 Add the following dependency to your `pom.xml` to use the Spargel.
 
-```xml
+~~~xml
 <dependency>
 	<groupId>org.apache.flink</groupId>
 	<artifactId>spargel</artifactId>
 	<version>{{site.FLINK_VERSION_STABLE}}</version>
 </dependency>
-```
+~~~
 
 Extend **VertexUpdateFunction&lt;***VertexKeyType*, *VertexValueType*, *MessageType***&gt;** to implement your *custom vertex update logic*.
 
@@ -40,7 +40,7 @@ Example: Propagate Minimum Vertex ID in Graph
 
 The Spargel operator **SpargelIteration** includes Spargel graph processing into your data flow. As usual, it can be combined with other operators like *map*, *reduce*, *join*, etc.
 
-```java
+~~~java
 FileDataSource vertices = new FileDataSource(...);
 FileDataSource edges = new FileDataSource(...);
 
@@ -53,7 +53,7 @@ FileDataSink result = new FileDataSink(...);
 result.setInput(iteration.getOutput());
 
 new Plan(result);
-```
+~~~
 
 Besides the **program logic** of vertex updates in *MinNeighborUpdater* and messages in *MinMessager*, you have to specify the **initial vertex** and **edge input**. Every vertex has a **key** and **value**. In each superstep, it **receives messages** from other vertices and updates its state:
 
@@ -68,7 +68,7 @@ For our example, we set the vertex ID as both *id and value* (initial minimum) a
 
 In order to **propagate the minimum vertex ID**, we iterate over all received messages (which contain the neighboring IDs) and update our value, if we found a new minimum:
 
-```java
+~~~java
 public class MinNeighborUpdater extends VertexUpdateFunction<IntValue, IntValue, IntValue> {
 	
 	@Override
@@ -87,11 +87,11 @@ public class MinNeighborUpdater extends VertexUpdateFunction<IntValue, IntValue,
 		}
 	}
 }
-```
+~~~
 
 The **messages in each superstep** consist of the **current minimum ID** seen by the vertex:
 
-```java
+~~~java
 public class MinMessager extends MessagingFunction<IntValue, IntValue, IntValue, NullValue> {
 	
 	@Override
@@ -100,7 +100,7 @@ public class MinMessager extends MessagingFunction<IntValue, IntValue, IntValue,
 		sendMessageToAllNeighbors(currentMin);
     }
 }
-```
+~~~
 
 The **API-provided method** `sendMessageToAllNeighbors(MessageType)` sends the message to all neighboring vertices. It is also possible to address specific vertices with `sendMessageTo(VertexKeyType, MessageType)`.
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/web_client.md
----------------------------------------------------------------------
diff --git a/docs/web_client.md b/docs/web_client.md
index ea844e0..6814271 100644
--- a/docs/web_client.md
+++ b/docs/web_client.md
@@ -2,9 +2,12 @@
 title:  "Web Client"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Flink provides a web interface to upload jobs, inspect their execution plans, and execute them. The interface is a great tool to showcase programs, debug execution plans, or demonstrate the system as a whole.
 
-# Start, Stop, and Configure the Web Interface
+## Starting, Stopping, and Configuring the Web Interface
 
 Start the web interface by executing:
 
@@ -16,14 +19,14 @@ and stop it by calling:
 
 The web interface runs on port 8080 by default. To specify a custom port set the ```webclient.port``` property in the *./conf/flink.yaml* configuration file. Jobs are submitted to the JobManager specified by ```jobmanager.rpc.address``` and ```jobmanager.rpc.port```. Please consult the [configuration](config.html#web_frontend) page for details and further configuration options.
 
-# Use the Web Interface
+## Using the Web Interface
 
 The web interface provides two views:
 
 1.  The **job view** to upload, preview, and submit Flink programs.
 2.  The **plan view** to analyze the optimized execution plans of Flink programs.
 
-## Job View
+### Job View
 
 The interface starts serving the job view. 
 
@@ -41,7 +44,7 @@ In case the jar's manifest file does not specify the program class, you can spec
 assembler <assemblerClass> <programArgs...>
 ```
 
-## Plan View
+### Plan View
 
 The plan view shows the optimized execution plan of the submitted program in the upper half of the page. The bottom part of the page displays detailed information about the currently selected plan operator including:
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/yarn_setup.md
----------------------------------------------------------------------
diff --git a/docs/yarn_setup.md b/docs/yarn_setup.md
index 57ef4b3..6078595 100644
--- a/docs/yarn_setup.md
+++ b/docs/yarn_setup.md
@@ -2,20 +2,23 @@
 title:  "YARN Setup"
 ---
 
-# In a Nutshell
+* This will be replaced by the TOC
+{:toc}
+
+## In a Nutshell
 
 Start YARN session with 4 Task Managers (each with 4 GB of Heapspace):
 
-```bash
+~~~bash
 wget {{ site.FLINK_DOWNLOAD_URL_YARN_STABLE }}
 tar xvzf flink-{{ site.FLINK_VERSION_STABLE }}-bin-hadoop2-yarn.tgz
 cd flink-yarn-{{ site.FLINK_VERSION_STABLE }}/
 ./bin/yarn-session.sh -n 4 -jm 1024 -tm 4096
-```
+~~~
 
 Specify the `-s` flag for the number of processing slots per Task Manager. We recommend to set the number of slots to the number of processors per machine.
 
-# Introducing YARN
+## Introducing YARN
 
 Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management framework. It allows to run various distributed applications on top of a cluster. Flink runs on YARN next to other applications. Users do not have to setup or install anything if there is already a YARN setup.
 
@@ -26,13 +29,13 @@ Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management
 
 If you have troubles using the Flink YARN client, have a look in the [FAQ section]({{site.baseurl}}/docs/0.5/general/faq.html).
 
-## Start Flink Session
+### Start Flink Session
 
 Follow these instructions to learn how to launch a Flink Session within your YARN cluster.
 
 A session will start all required Flink services (JobManager and TaskManagers) so that you can submit programs to the cluster. Note that you can run multiple programs per session.
 
-### Download Flink for YARN
+#### Download Flink for YARN
 
 Download the YARN tgz package on the [download page]({{site.baseurl}}/downloads/). It contains the required files.
 
@@ -41,22 +44,22 @@ If you want to build the YARN .tgz file from sources, follow the [build instruct
 
 Extract the package using:
 
-```bash
+~~~bash
 tar xvzf flink-dist-{{site.FLINK_VERSION_STABLE }}-yarn.tar.gz
 cd flink-yarn-{{site.FLINK_VERSION_STABLE }}/
-```
+~~~
 
-### Start a Session
+#### Start a Session
 
 Use the following command to start a session
 
-```bash
+~~~bash
 ./bin/yarn-session.sh
-```
+~~~
 
 This command will show you the following overview:
 
-```bash
+~~~bash
 Usage:
    Required
      -n,--container <arg>   Number of Yarn container to allocate (=Number of Task Managers)
@@ -69,15 +72,15 @@ Usage:
      -tm,--taskManagerMemory <arg>   Memory per TaskManager Container [in MB]
      -tmc,--taskManagerCores <arg>   Virtual CPU cores per TaskManager
      -v,--verbose                    Verbose debug mode
-```
+~~~
 
 Please note that the Client requires the `HADOOP_HOME` (or `YARN_CONF_DIR` or `HADOOP_CONF_DIR`) environment variable to be set to read the YARN and HDFS configuration.
 
 **Example:** Issue the following command to allocate 10 Task Managers, with 8 GB of memory and 32 processing slots each:
 
-```bash
+~~~bash
 ./bin/yarn-session.sh -n 10 -tm 8192 -s 32
-```
+~~~
 
 The system will use the configuration in `conf/flink-config.yaml`. Please follow our [configuration guide](config.html) if you want to change something. 
 
@@ -97,19 +100,19 @@ The client has to remain open to keep the deployment running. We suggest to use
 4. Use `screen -r` to resume again.
 
 
-# Submit Job to Flink
+## Submit Job to Flink
 
 Use the following command to submit a Flink program to the YARN cluster:
 
-```bash
+~~~bash
 ./bin/flink
-```
+~~~
 
 Please refer to the documentation of the [commandline client](cli.html).
 
 The command will show you a help menu like this:
 
-```bash
+~~~bash
 [...]
 Action "run" compiles and runs a program.
 
@@ -127,48 +130,48 @@ Action "run" compiles and runs a program.
                                       program. Optional flag to override the
                                       default value specified in the
                                       configuration
-```
+~~~
 
 Use the *run* action to submit a job to YARN. The client is able to determine the address of the JobManager. In the rare event of a problem, you can also pass the JobManager address using the `-m` argument. The JobManager address is visible in the YARN console.
 
 **Example**
 
-```bash
+~~~bash
 wget -O apache-license-v2.txt http://www.apache.org/licenses/LICENSE-2.0.txt
 
 ./bin/flink run -j ./examples/flink-java-examples-{{site.FLINK_VERSION_STABLE }}-WordCount.jar \
                        -a 1 file://`pwd`/apache-license-v2.txt file://`pwd`/wordcount-result.txt 
-```
+~~~
 
 If there is the following error, make sure that all TaskManagers started:
 
-```bash
+~~~bash
 Exception in thread "main" org.apache.flinkcompiler.CompilerException:
     Available instances could not be determined from job manager: Connection timed out.
-```
+~~~
 
 You can check the number of TaskManagers in the JobManager web interface. The address of this interface is printed in the YARN session console.
 
 If the TaskManagers do not show up after a minute, you should investigate the issue using the log files.
 
 
-# Debugging a failed YARN session
+## Debugging a failed YARN session
 
 There are many reasons why a Flink YARN session deployment can fail. A misconfigured Hadoop setup (HDFS permissions, YARN configuration), version incompatibilities (running Flink with vanilla Hadoop dependencies on Cloudera Hadoop) or other errors.
 
-## Log Files
+### Log Files
 
 In cases where the Flink YARN session fails during the deployment itself, users have to rely on the logging capabilities of Hadoop YARN. The most useful feature for that is the [YARN log aggregation](http://hortonworks.com/blog/simplifying-user-logs-management-and-access-in-yarn/). 
 To enable it, users have to set the `yarn.log-aggregation-enable` property to `true` in the `yarn-site.xml` file.
 Once that is enabled, users can use the following command to retrieve all log files of a (failed) YARN session.
 
-```
+~~~
 yarn logs -applicationId <application ID>
-```
+~~~
 
 Note that it takes a few seconds after the session has finished until the logs show up.
 
-## YARN Client console & Webinterfaces
+### YARN Client console & Webinterfaces
 
 The Flink YARN client also prints error messages in the terminal if errors occur during runtime (for example if a TaskManager stops working after some time).
 
@@ -177,12 +180,12 @@ In addition to that, there is the YARN Resource Manager webinterface (by default
 It allows to access log files for running YARN applications and shows diagnostics for failed apps.
 
 
-# Build YARN client for a specific Hadoop version
+## Build YARN client for a specific Hadoop version
 
 Users using Hadoop distributions from companies like Hortonworks, Cloudera or MapR might have to build Flink against their specific versions of Hadoop (HDFS) and YARN. Please read the [build instructions](building.html) for more details.
 
 
-# Background
+## Background
 
 This section briefly describes how Flink and YARN interact. 
 


[55/60] Turn Documentation into standalone website, add Overview Page

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/css/bootstrap-lumen-custom.css
----------------------------------------------------------------------
diff --git a/docs/css/bootstrap-lumen-custom.css b/docs/css/bootstrap-lumen-custom.css
new file mode 100644
index 0000000..dc9ecc8
--- /dev/null
+++ b/docs/css/bootstrap-lumen-custom.css
@@ -0,0 +1,6189 @@
+@import url("//fonts.googleapis.com/css?family=Source+Sans+Pro:300,400,700,400italic");
+/*!
+ * Bootswatch v3.1.1+1
+ * Homepage: http://bootswatch.com
+ * Copyright 2012-2014 Thomas Park
+ * Licensed under MIT
+ * Based on Bootstrap
+*/
+/*! normalize.css v3.0.0 | MIT License | git.io/normalize */
+html {
+  font-family: sans-serif;
+  -ms-text-size-adjust: 100%;
+  -webkit-text-size-adjust: 100%;
+}
+body {
+  margin: 0;
+}
+article,
+aside,
+details,
+figcaption,
+figure,
+footer,
+header,
+hgroup,
+main,
+nav,
+section,
+summary {
+  display: block;
+}
+audio,
+canvas,
+progress,
+video {
+  display: inline-block;
+  vertical-align: baseline;
+}
+audio:not([controls]) {
+  display: none;
+  height: 0;
+}
+[hidden],
+template {
+  display: none;
+}
+a {
+  background: transparent;
+}
+a:active,
+a:hover {
+  outline: 0;
+}
+abbr[title] {
+  border-bottom: 1px dotted;
+}
+b,
+strong {
+  font-weight: bold;
+}
+dfn {
+  font-style: italic;
+}
+h1 {
+  font-size: 2em;
+  margin: 0.67em 0;
+}
+mark {
+  background: #ff0;
+  color: #000;
+}
+small {
+  font-size: 80%;
+}
+sub,
+sup {
+  font-size: 75%;
+  line-height: 0;
+  position: relative;
+  vertical-align: baseline;
+}
+sup {
+  top: -0.5em;
+}
+sub {
+  bottom: -0.25em;
+}
+img {
+  border: 0;
+}
+svg:not(:root) {
+  overflow: hidden;
+}
+figure {
+  margin: 1em 40px;
+}
+hr {
+  -moz-box-sizing: content-box;
+  box-sizing: content-box;
+  height: 0;
+}
+pre {
+  overflow: auto;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: monospace, monospace;
+  font-size: 1em;
+}
+button,
+input,
+optgroup,
+select,
+textarea {
+  color: inherit;
+  font: inherit;
+  margin: 0;
+}
+button {
+  overflow: visible;
+}
+button,
+select {
+  text-transform: none;
+}
+button,
+html input[type="button"],
+input[type="reset"],
+input[type="submit"] {
+  -webkit-appearance: button;
+  cursor: pointer;
+}
+button[disabled],
+html input[disabled] {
+  cursor: default;
+}
+button::-moz-focus-inner,
+input::-moz-focus-inner {
+  border: 0;
+  padding: 0;
+}
+input {
+  line-height: normal;
+}
+input[type="checkbox"],
+input[type="radio"] {
+  box-sizing: border-box;
+  padding: 0;
+}
+input[type="number"]::-webkit-inner-spin-button,
+input[type="number"]::-webkit-outer-spin-button {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: textfield;
+  -moz-box-sizing: content-box;
+  -webkit-box-sizing: content-box;
+  box-sizing: content-box;
+}
+input[type="search"]::-webkit-search-cancel-button,
+input[type="search"]::-webkit-search-decoration {
+  -webkit-appearance: none;
+}
+fieldset {
+  border: 1px solid #c0c0c0;
+  margin: 0 2px;
+  padding: 0.35em 0.625em 0.75em;
+}
+legend {
+  border: 0;
+  padding: 0;
+}
+textarea {
+  overflow: auto;
+}
+optgroup {
+  font-weight: bold;
+}
+table {
+  border-collapse: collapse;
+  border-spacing: 0;
+}
+td,
+th {
+  padding: 0;
+}
+@media print {
+  * {
+    text-shadow: none !important;
+    color: #000 !important;
+    background: transparent !important;
+    box-shadow: none !important;
+  }
+  a,
+  a:visited {
+    text-decoration: underline;
+  }
+  a[href]:after {
+    content: " (" attr(href) ")";
+  }
+  abbr[title]:after {
+    content: " (" attr(title) ")";
+  }
+  a[href^="javascript:"]:after,
+  a[href^="#"]:after {
+    content: "";
+  }
+  pre,
+  blockquote {
+    border: 1px solid #999;
+    page-break-inside: avoid;
+  }
+  thead {
+    display: table-header-group;
+  }
+  tr,
+  img {
+    page-break-inside: avoid;
+  }
+  img {
+    max-width: 100% !important;
+  }
+  p,
+  h2,
+  h3 {
+    orphans: 3;
+    widows: 3;
+  }
+  h2,
+  h3 {
+    page-break-after: avoid;
+  }
+  select {
+    background: #fff !important;
+  }
+  .navbar {
+    display: none;
+  }
+  .table td,
+  .table th {
+    background-color: #fff !important;
+  }
+  .btn > .caret,
+  .dropup > .btn > .caret {
+    border-top-color: #000 !important;
+  }
+  .label {
+    border: 1px solid #000;
+  }
+  .table {
+    border-collapse: collapse !important;
+  }
+  .table-bordered th,
+  .table-bordered td {
+    border: 1px solid #ddd !important;
+  }
+}
+* {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+*:before,
+*:after {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+html {
+  font-size: 62.5%;
+  -webkit-tap-highlight-color: rgba(0, 0, 0, 0);
+}
+body {
+  font-family: "Source Sans Pro", "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: 18px;
+  line-height: 1.42857143;
+  color: #555555;
+  background-color: #ffffff;
+}
+input,
+button,
+select,
+textarea {
+  font-family: inherit;
+  font-size: inherit;
+  line-height: inherit;
+}
+a {
+  color: #158cba;
+  text-decoration: none;
+}
+a:hover,
+a:focus {
+  color: #158cba;
+  text-decoration: underline;
+}
+a:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+figure {
+  margin: 0;
+}
+img {
+  vertical-align: middle;
+}
+.img-responsive,
+.thumbnail > img,
+.thumbnail a > img,
+.carousel-inner > .item > img,
+.carousel-inner > .item > a > img {
+  display: block;
+  max-width: 100%;
+  height: auto;
+}
+.img-rounded {
+  border-radius: 5px;
+}
+.img-thumbnail {
+  padding: 4px;
+  line-height: 1.42857143;
+  background-color: #ffffff;
+  border: 1px solid #eeeeee;
+  border-radius: 4px;
+  -webkit-transition: all 0.2s ease-in-out;
+  transition: all 0.2s ease-in-out;
+  display: inline-block;
+  max-width: 100%;
+  height: auto;
+}
+.img-circle {
+  border-radius: 50%;
+}
+hr {
+  margin-top: 25px;
+  margin-bottom: 25px;
+  border: 0;
+  border-top: 1px solid #eeeeee;
+}
+.sr-only {
+  position: absolute;
+  width: 1px;
+  height: 1px;
+  margin: -1px;
+  padding: 0;
+  overflow: hidden;
+  clip: rect(0, 0, 0, 0);
+  border: 0;
+}
+h1,
+h2,
+h3,
+h4,
+h5,
+h6,
+.h1,
+.h2,
+.h3,
+.h4,
+.h5,
+.h6 {
+  font-family: inherit;
+  font-weight: 400;
+  line-height: 1.1;
+  color: #333333;
+}
+h1 small,
+h2 small,
+h3 small,
+h4 small,
+h5 small,
+h6 small,
+.h1 small,
+.h2 small,
+.h3 small,
+.h4 small,
+.h5 small,
+.h6 small,
+h1 .small,
+h2 .small,
+h3 .small,
+h4 .small,
+h5 .small,
+h6 .small,
+.h1 .small,
+.h2 .small,
+.h3 .small,
+.h4 .small,
+.h5 .small,
+.h6 .small {
+  font-weight: normal;
+  line-height: 1;
+  color: #999999;
+}
+h1,
+.h1,
+h2,
+.h2,
+h3,
+.h3 {
+  margin-top: 25px;
+  margin-bottom: 12.5px;
+}
+h1 small,
+.h1 small,
+h2 small,
+.h2 small,
+h3 small,
+.h3 small,
+h1 .small,
+.h1 .small,
+h2 .small,
+.h2 .small,
+h3 .small,
+.h3 .small {
+  font-size: 65%;
+}
+h4,
+.h4,
+h5,
+.h5,
+h6,
+.h6 {
+  margin-top: 12.5px;
+  margin-bottom: 12.5px;
+}
+h4 small,
+.h4 small,
+h5 small,
+.h5 small,
+h6 small,
+.h6 small,
+h4 .small,
+.h4 .small,
+h5 .small,
+.h5 .small,
+h6 .small,
+.h6 .small {
+  font-size: 75%;
+}
+h1,
+.h1 {
+  font-size: 43px;
+}
+h2,
+.h2 {
+  font-size: 38px;
+}
+h3,
+.h3 {
+  font-size: 31px;
+}
+h4,
+.h4 {
+  font-size: 23px;
+}
+h5,
+.h5 {
+  font-size: 18px;
+}
+h6,
+.h6 {
+  font-size: 16px;
+}
+p {
+  margin: 0 0 12.5px;
+}
+.lead {
+  margin-bottom: 25px;
+  font-size: 20px;
+  font-weight: 200;
+  line-height: 1.4;
+}
+@media (min-width: 768px) {
+  .lead {
+    font-size: 27px;
+  }
+}
+small,
+.small {
+  font-size: 85%;
+}
+cite {
+  font-style: normal;
+}
+.text-left {
+  text-align: left;
+}
+.text-right {
+  text-align: right;
+}
+.text-center {
+  text-align: center;
+}
+.text-justify {
+  text-align: justify;
+}
+.text-muted {
+  color: #999999;
+}
+.text-primary {
+  color: #158cba;
+}
+a.text-primary:hover {
+  color: #106a8c;
+}
+.text-success {
+  color: #ffffff;
+}
+a.text-success:hover {
+  color: #e6e6e6;
+}
+.text-info {
+  color: #ffffff;
+}
+a.text-info:hover {
+  color: #e6e6e6;
+}
+.text-warning {
+  color: #ffffff;
+}
+a.text-warning:hover {
+  color: #e6e6e6;
+}
+.text-danger {
+  color: #ffffff;
+}
+a.text-danger:hover {
+  color: #e6e6e6;
+}
+.bg-primary {
+  color: #fff;
+  background-color: #158cba;
+}
+a.bg-primary:hover {
+  background-color: #106a8c;
+}
+.bg-success {
+  background-color: #28b62c;
+}
+a.bg-success:hover {
+  background-color: #1f8c22;
+}
+.bg-info {
+  background-color: #75caeb;
+}
+a.bg-info:hover {
+  background-color: #48b9e5;
+}
+.bg-warning {
+  background-color: #ff851b;
+}
+a.bg-warning:hover {
+  background-color: #e76b00;
+}
+.bg-danger {
+  background-color: #ff4136;
+}
+a.bg-danger:hover {
+  background-color: #ff1103;
+}
+.page-header {
+  padding-bottom: 11.5px;
+  margin: 50px 0 25px;
+  border-bottom: 1px solid #eeeeee;
+}
+ul,
+ol {
+  margin-top: 0;
+  margin-bottom: 12.5px;
+}
+ul ul,
+ol ul,
+ul ol,
+ol ol {
+  margin-bottom: 0;
+}
+.list-unstyled {
+  padding-left: 0;
+  list-style: none;
+}
+.list-inline {
+  padding-left: 0;
+  list-style: none;
+  margin-left: -5px;
+}
+.list-inline > li {
+  display: inline-block;
+  padding-left: 5px;
+  padding-right: 5px;
+}
+dl {
+  margin-top: 0;
+  margin-bottom: 25px;
+}
+dt,
+dd {
+  line-height: 1.42857143;
+}
+dt {
+  font-weight: bold;
+}
+dd {
+  margin-left: 0;
+}
+@media (min-width: 768px) {
+  .dl-horizontal dt {
+    float: left;
+    width: 160px;
+    clear: left;
+    text-align: right;
+    overflow: hidden;
+    text-overflow: ellipsis;
+    white-space: nowrap;
+  }
+  .dl-horizontal dd {
+    margin-left: 180px;
+  }
+}
+abbr[title],
+abbr[data-original-title] {
+  cursor: help;
+  border-bottom: 1px dotted #999999;
+}
+.initialism {
+  font-size: 90%;
+  text-transform: uppercase;
+}
+blockquote {
+  padding: 12.5px 25px;
+  margin: 0 0 25px;
+  font-size: 22.5px;
+  border-left: 5px solid #eeeeee;
+}
+blockquote p:last-child,
+blockquote ul:last-child,
+blockquote ol:last-child {
+  margin-bottom: 0;
+}
+blockquote footer,
+blockquote small,
+blockquote .small {
+  display: block;
+  font-size: 80%;
+  line-height: 1.42857143;
+  color: #999999;
+}
+blockquote footer:before,
+blockquote small:before,
+blockquote .small:before {
+  content: '\2014 \00A0';
+}
+.blockquote-reverse,
+blockquote.pull-right {
+  padding-right: 15px;
+  padding-left: 0;
+  border-right: 5px solid #eeeeee;
+  border-left: 0;
+  text-align: right;
+}
+.blockquote-reverse footer:before,
+blockquote.pull-right footer:before,
+.blockquote-reverse small:before,
+blockquote.pull-right small:before,
+.blockquote-reverse .small:before,
+blockquote.pull-right .small:before {
+  content: '';
+}
+.blockquote-reverse footer:after,
+blockquote.pull-right footer:after,
+.blockquote-reverse small:after,
+blockquote.pull-right small:after,
+.blockquote-reverse .small:after,
+blockquote.pull-right .small:after {
+  content: '\00A0 \2014';
+}
+blockquote:before,
+blockquote:after {
+  content: "";
+}
+address {
+  margin-bottom: 25px;
+  font-style: normal;
+  line-height: 1.42857143;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: Menlo, Monaco, Consolas, "Courier New", monospace;
+}
+code {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #c7254e;
+  background-color: #f9f2f4;
+  white-space: nowrap;
+  border-radius: 4px;
+}
+kbd {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #ffffff;
+  background-color: #333333;
+  border-radius: 2px;
+  box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);
+}
+pre {
+  display: block;
+  padding: 12px;
+  margin: 0 0 12.5px;
+  font-size: 17px;
+  line-height: 1.42857143;
+  word-break: break-all;
+  word-wrap: break-word;
+  color: #333333;
+  background-color: #f5f5f5;
+  border: 1px solid #cccccc;
+  border-radius: 4px;
+}
+pre code {
+  padding: 0;
+  font-size: inherit;
+  color: inherit;
+  white-space: pre-wrap;
+  background-color: transparent;
+  border-radius: 0;
+}
+.pre-scrollable {
+  max-height: 340px;
+  overflow-y: scroll;
+}
+.container {
+  margin-right: auto;
+  margin-left: auto;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+@media (min-width: 768px) {
+  .container {
+    width: 750px;
+  }
+}
+@media (min-width: 992px) {
+  .container {
+    width: 970px;
+  }
+}
+@media (min-width: 1200px) {
+  .container {
+    width: 1170px;
+  }
+}
+.container-fluid {
+  margin-right: auto;
+  margin-left: auto;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+.row {
+  margin-left: -15px;
+  margin-right: -15px;
+}
+.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {
+  position: relative;
+  min-height: 1px;
+  padding-left: 15px;
+  padding-right: 15px;
+}
+.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {
+  float: left;
+}
+.col-xs-12 {
+  width: 100%;
+}
+.col-xs-11 {
+  width: 91.66666667%;
+}
+.col-xs-10 {
+  width: 83.33333333%;
+}
+.col-xs-9 {
+  width: 75%;
+}
+.col-xs-8 {
+  width: 66.66666667%;
+}
+.col-xs-7 {
+  width: 58.33333333%;
+}
+.col-xs-6 {
+  width: 50%;
+}
+.col-xs-5 {
+  width: 41.66666667%;
+}
+.col-xs-4 {
+  width: 33.33333333%;
+}
+.col-xs-3 {
+  width: 25%;
+}
+.col-xs-2 {
+  width: 16.66666667%;
+}
+.col-xs-1 {
+  width: 8.33333333%;
+}
+.col-xs-pull-12 {
+  right: 100%;
+}
+.col-xs-pull-11 {
+  right: 91.66666667%;
+}
+.col-xs-pull-10 {
+  right: 83.33333333%;
+}
+.col-xs-pull-9 {
+  right: 75%;
+}
+.col-xs-pull-8 {
+  right: 66.66666667%;
+}
+.col-xs-pull-7 {
+  right: 58.33333333%;
+}
+.col-xs-pull-6 {
+  right: 50%;
+}
+.col-xs-pull-5 {
+  right: 41.66666667%;
+}
+.col-xs-pull-4 {
+  right: 33.33333333%;
+}
+.col-xs-pull-3 {
+  right: 25%;
+}
+.col-xs-pull-2 {
+  right: 16.66666667%;
+}
+.col-xs-pull-1 {
+  right: 8.33333333%;
+}
+.col-xs-pull-0 {
+  right: 0%;
+}
+.col-xs-push-12 {
+  left: 100%;
+}
+.col-xs-push-11 {
+  left: 91.66666667%;
+}
+.col-xs-push-10 {
+  left: 83.33333333%;
+}
+.col-xs-push-9 {
+  left: 75%;
+}
+.col-xs-push-8 {
+  left: 66.66666667%;
+}
+.col-xs-push-7 {
+  left: 58.33333333%;
+}
+.col-xs-push-6 {
+  left: 50%;
+}
+.col-xs-push-5 {
+  left: 41.66666667%;
+}
+.col-xs-push-4 {
+  left: 33.33333333%;
+}
+.col-xs-push-3 {
+  left: 25%;
+}
+.col-xs-push-2 {
+  left: 16.66666667%;
+}
+.col-xs-push-1 {
+  left: 8.33333333%;
+}
+.col-xs-push-0 {
+  left: 0%;
+}
+.col-xs-offset-12 {
+  margin-left: 100%;
+}
+.col-xs-offset-11 {
+  margin-left: 91.66666667%;
+}
+.col-xs-offset-10 {
+  margin-left: 83.33333333%;
+}
+.col-xs-offset-9 {
+  margin-left: 75%;
+}
+.col-xs-offset-8 {
+  margin-left: 66.66666667%;
+}
+.col-xs-offset-7 {
+  margin-left: 58.33333333%;
+}
+.col-xs-offset-6 {
+  margin-left: 50%;
+}
+.col-xs-offset-5 {
+  margin-left: 41.66666667%;
+}
+.col-xs-offset-4 {
+  margin-left: 33.33333333%;
+}
+.col-xs-offset-3 {
+  margin-left: 25%;
+}
+.col-xs-offset-2 {
+  margin-left: 16.66666667%;
+}
+.col-xs-offset-1 {
+  margin-left: 8.33333333%;
+}
+.col-xs-offset-0 {
+  margin-left: 0%;
+}
+@media (min-width: 768px) {
+  .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {
+    float: left;
+  }
+  .col-sm-12 {
+    width: 100%;
+  }
+  .col-sm-11 {
+    width: 91.66666667%;
+  }
+  .col-sm-10 {
+    width: 83.33333333%;
+  }
+  .col-sm-9 {
+    width: 75%;
+  }
+  .col-sm-8 {
+    width: 66.66666667%;
+  }
+  .col-sm-7 {
+    width: 58.33333333%;
+  }
+  .col-sm-6 {
+    width: 50%;
+  }
+  .col-sm-5 {
+    width: 41.66666667%;
+  }
+  .col-sm-4 {
+    width: 33.33333333%;
+  }
+  .col-sm-3 {
+    width: 25%;
+  }
+  .col-sm-2 {
+    width: 16.66666667%;
+  }
+  .col-sm-1 {
+    width: 8.33333333%;
+  }
+  .col-sm-pull-12 {
+    right: 100%;
+  }
+  .col-sm-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-sm-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-sm-pull-9 {
+    right: 75%;
+  }
+  .col-sm-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-sm-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-sm-pull-6 {
+    right: 50%;
+  }
+  .col-sm-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-sm-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-sm-pull-3 {
+    right: 25%;
+  }
+  .col-sm-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-sm-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-sm-pull-0 {
+    right: 0%;
+  }
+  .col-sm-push-12 {
+    left: 100%;
+  }
+  .col-sm-push-11 {
+    left: 91.66666667%;
+  }
+  .col-sm-push-10 {
+    left: 83.33333333%;
+  }
+  .col-sm-push-9 {
+    left: 75%;
+  }
+  .col-sm-push-8 {
+    left: 66.66666667%;
+  }
+  .col-sm-push-7 {
+    left: 58.33333333%;
+  }
+  .col-sm-push-6 {
+    left: 50%;
+  }
+  .col-sm-push-5 {
+    left: 41.66666667%;
+  }
+  .col-sm-push-4 {
+    left: 33.33333333%;
+  }
+  .col-sm-push-3 {
+    left: 25%;
+  }
+  .col-sm-push-2 {
+    left: 16.66666667%;
+  }
+  .col-sm-push-1 {
+    left: 8.33333333%;
+  }
+  .col-sm-push-0 {
+    left: 0%;
+  }
+  .col-sm-offset-12 {
+    margin-left: 100%;
+  }
+  .col-sm-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-sm-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-sm-offset-9 {
+    margin-left: 75%;
+  }
+  .col-sm-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-sm-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-sm-offset-6 {
+    margin-left: 50%;
+  }
+  .col-sm-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-sm-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-sm-offset-3 {
+    margin-left: 25%;
+  }
+  .col-sm-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-sm-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-sm-offset-0 {
+    margin-left: 0%;
+  }
+}
+@media (min-width: 992px) {
+  .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {
+    float: left;
+  }
+  .col-md-12 {
+    width: 100%;
+  }
+  .col-md-11 {
+    width: 91.66666667%;
+  }
+  .col-md-10 {
+    width: 83.33333333%;
+  }
+  .col-md-9 {
+    width: 75%;
+  }
+  .col-md-8 {
+    width: 66.66666667%;
+  }
+  .col-md-7 {
+    width: 58.33333333%;
+  }
+  .col-md-6 {
+    width: 50%;
+  }
+  .col-md-5 {
+    width: 41.66666667%;
+  }
+  .col-md-4 {
+    width: 33.33333333%;
+  }
+  .col-md-3 {
+    width: 25%;
+  }
+  .col-md-2 {
+    width: 16.66666667%;
+  }
+  .col-md-1 {
+    width: 8.33333333%;
+  }
+  .col-md-pull-12 {
+    right: 100%;
+  }
+  .col-md-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-md-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-md-pull-9 {
+    right: 75%;
+  }
+  .col-md-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-md-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-md-pull-6 {
+    right: 50%;
+  }
+  .col-md-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-md-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-md-pull-3 {
+    right: 25%;
+  }
+  .col-md-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-md-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-md-pull-0 {
+    right: 0%;
+  }
+  .col-md-push-12 {
+    left: 100%;
+  }
+  .col-md-push-11 {
+    left: 91.66666667%;
+  }
+  .col-md-push-10 {
+    left: 83.33333333%;
+  }
+  .col-md-push-9 {
+    left: 75%;
+  }
+  .col-md-push-8 {
+    left: 66.66666667%;
+  }
+  .col-md-push-7 {
+    left: 58.33333333%;
+  }
+  .col-md-push-6 {
+    left: 50%;
+  }
+  .col-md-push-5 {
+    left: 41.66666667%;
+  }
+  .col-md-push-4 {
+    left: 33.33333333%;
+  }
+  .col-md-push-3 {
+    left: 25%;
+  }
+  .col-md-push-2 {
+    left: 16.66666667%;
+  }
+  .col-md-push-1 {
+    left: 8.33333333%;
+  }
+  .col-md-push-0 {
+    left: 0%;
+  }
+  .col-md-offset-12 {
+    margin-left: 100%;
+  }
+  .col-md-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-md-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-md-offset-9 {
+    margin-left: 75%;
+  }
+  .col-md-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-md-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-md-offset-6 {
+    margin-left: 50%;
+  }
+  .col-md-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-md-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-md-offset-3 {
+    margin-left: 25%;
+  }
+  .col-md-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-md-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-md-offset-0 {
+    margin-left: 0%;
+  }
+}
+@media (min-width: 1200px) {
+  .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
+    float: left;
+  }
+  .col-lg-12 {
+    width: 100%;
+  }
+  .col-lg-11 {
+    width: 91.66666667%;
+  }
+  .col-lg-10 {
+    width: 83.33333333%;
+  }
+  .col-lg-9 {
+    width: 75%;
+  }
+  .col-lg-8 {
+    width: 66.66666667%;
+  }
+  .col-lg-7 {
+    width: 58.33333333%;
+  }
+  .col-lg-6 {
+    width: 50%;
+  }
+  .col-lg-5 {
+    width: 41.66666667%;
+  }
+  .col-lg-4 {
+    width: 33.33333333%;
+  }
+  .col-lg-3 {
+    width: 25%;
+  }
+  .col-lg-2 {
+    width: 16.66666667%;
+  }
+  .col-lg-1 {
+    width: 8.33333333%;
+  }
+  .col-lg-pull-12 {
+    right: 100%;
+  }
+  .col-lg-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-lg-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-lg-pull-9 {
+    right: 75%;
+  }
+  .col-lg-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-lg-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-lg-pull-6 {
+    right: 50%;
+  }
+  .col-lg-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-lg-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-lg-pull-3 {
+    right: 25%;
+  }
+  .col-lg-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-lg-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-lg-pull-0 {
+    right: 0%;
+  }
+  .col-lg-push-12 {
+    left: 100%;
+  }
+  .col-lg-push-11 {
+    left: 91.66666667%;
+  }
+  .col-lg-push-10 {
+    left: 83.33333333%;
+  }
+  .col-lg-push-9 {
+    left: 75%;
+  }
+  .col-lg-push-8 {
+    left: 66.66666667%;
+  }
+  .col-lg-push-7 {
+    left: 58.33333333%;
+  }
+  .col-lg-push-6 {
+    left: 50%;
+  }
+  .col-lg-push-5 {
+    left: 41.66666667%;
+  }
+  .col-lg-push-4 {
+    left: 33.33333333%;
+  }
+  .col-lg-push-3 {
+    left: 25%;
+  }
+  .col-lg-push-2 {
+    left: 16.66666667%;
+  }
+  .col-lg-push-1 {
+    left: 8.33333333%;
+  }
+  .col-lg-push-0 {
+    left: 0%;
+  }
+  .col-lg-offset-12 {
+    margin-left: 100%;
+  }
+  .col-lg-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-lg-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-lg-offset-9 {
+    margin-left: 75%;
+  }
+  .col-lg-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-lg-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-lg-offset-6 {
+    margin-left: 50%;
+  }
+  .col-lg-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-lg-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-lg-offset-3 {
+    margin-left: 25%;
+  }
+  .col-lg-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-lg-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-lg-offset-0 {
+    margin-left: 0%;
+  }
+}
+table {
+  max-width: 100%;
+  background-color: transparent;
+}
+th {
+  text-align: left;
+}
+.table {
+  width: 100%;
+  margin-bottom: 25px;
+}
+.table > thead > tr > th,
+.table > tbody > tr > th,
+.table > tfoot > tr > th,
+.table > thead > tr > td,
+.table > tbody > tr > td,
+.table > tfoot > tr > td {
+  padding: 8px;
+  line-height: 1.42857143;
+  vertical-align: top;
+  border-top: 1px solid #dddddd;
+}
+.table > thead > tr > th {
+  vertical-align: bottom;
+  border-bottom: 2px solid #dddddd;
+}
+.table > caption + thead > tr:first-child > th,
+.table > colgroup + thead > tr:first-child > th,
+.table > thead:first-child > tr:first-child > th,
+.table > caption + thead > tr:first-child > td,
+.table > colgroup + thead > tr:first-child > td,
+.table > thead:first-child > tr:first-child > td {
+  border-top: 0;
+}
+.table > tbody + tbody {
+  border-top: 2px solid #dddddd;
+}
+.table .table {
+  background-color: #ffffff;
+}
+.table-condensed > thead > tr > th,
+.table-condensed > tbody > tr > th,
+.table-condensed > tfoot > tr > th,
+.table-condensed > thead > tr > td,
+.table-condensed > tbody > tr > td,
+.table-condensed > tfoot > tr > td {
+  padding: 5px;
+}
+.table-bordered {
+  border: 1px solid #dddddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > tbody > tr > th,
+.table-bordered > tfoot > tr > th,
+.table-bordered > thead > tr > td,
+.table-bordered > tbody > tr > td,
+.table-bordered > tfoot > tr > td {
+  border: 1px solid #dddddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > thead > tr > td {
+  border-bottom-width: 2px;
+}
+.table-striped > tbody > tr:nth-child(odd) > td,
+.table-striped > tbody > tr:nth-child(odd) > th {
+  background-color: #f9f9f9;
+}
+.table-hover > tbody > tr:hover > td,
+.table-hover > tbody > tr:hover > th {
+  background-color: #f5f5f5;
+}
+table col[class*="col-"] {
+  position: static;
+  float: none;
+  display: table-column;
+}
+table td[class*="col-"],
+table th[class*="col-"] {
+  position: static;
+  float: none;
+  display: table-cell;
+}
+.table > thead > tr > td.active,
+.table > tbody > tr > td.active,
+.table > tfoot > tr > td.active,
+.table > thead > tr > th.active,
+.table > tbody > tr > th.active,
+.table > tfoot > tr > th.active,
+.table > thead > tr.active > td,
+.table > tbody > tr.active > td,
+.table > tfoot > tr.active > td,
+.table > thead > tr.active > th,
+.table > tbody > tr.active > th,
+.table > tfoot > tr.active > th {
+  background-color: #f5f5f5;
+}
+.table-hover > tbody > tr > td.active:hover,
+.table-hover > tbody > tr > th.active:hover,
+.table-hover > tbody > tr.active:hover > td,
+.table-hover > tbody > tr.active:hover > th {
+  background-color: #e8e8e8;
+}
+.table > thead > tr > td.success,
+.table > tbody > tr > td.success,
+.table > tfoot > tr > td.success,
+.table > thead > tr > th.success,
+.table > tbody > tr > th.success,
+.table > tfoot > tr > th.success,
+.table > thead > tr.success > td,
+.table > tbody > tr.success > td,
+.table > tfoot > tr.success > td,
+.table > thead > tr.success > th,
+.table > tbody > tr.success > th,
+.table > tfoot > tr.success > th {
+  background-color: #28b62c;
+}
+.table-hover > tbody > tr > td.success:hover,
+.table-hover > tbody > tr > th.success:hover,
+.table-hover > tbody > tr.success:hover > td,
+.table-hover > tbody > tr.success:hover > th {
+  background-color: #23a127;
+}
+.table > thead > tr > td.info,
+.table > tbody > tr > td.info,
+.table > tfoot > tr > td.info,
+.table > thead > tr > th.info,
+.table > tbody > tr > th.info,
+.table > tfoot > tr > th.info,
+.table > thead > tr.info > td,
+.table > tbody > tr.info > td,
+.table > tfoot > tr.info > td,
+.table > thead > tr.info > th,
+.table > tbody > tr.info > th,
+.table > tfoot > tr.info > th {
+  background-color: #75caeb;
+}
+.table-hover > tbody > tr > td.info:hover,
+.table-hover > tbody > tr > th.info:hover,
+.table-hover > tbody > tr.info:hover > td,
+.table-hover > tbody > tr.info:hover > th {
+  background-color: #5fc1e8;
+}
+.table > thead > tr > td.warning,
+.table > tbody > tr > td.warning,
+.table > tfoot > tr > td.warning,
+.table > thead > tr > th.warning,
+.table > tbody > tr > th.warning,
+.table > tfoot > tr > th.warning,
+.table > thead > tr.warning > td,
+.table > tbody > tr.warning > td,
+.table > tfoot > tr.warning > td,
+.table > thead > tr.warning > th,
+.table > tbody > tr.warning > th,
+.table > tfoot > tr.warning > th {
+  background-color: #ff851b;
+}
+.table-hover > tbody > tr > td.warning:hover,
+.table-hover > tbody > tr > th.warning:hover,
+.table-hover > tbody > tr.warning:hover > td,
+.table-hover > tbody > tr.warning:hover > th {
+  background-color: #ff7701;
+}
+.table > thead > tr > td.danger,
+.table > tbody > tr > td.danger,
+.table > tfoot > tr > td.danger,
+.table > thead > tr > th.danger,
+.table > tbody > tr > th.danger,
+.table > tfoot > tr > th.danger,
+.table > thead > tr.danger > td,
+.table > tbody > tr.danger > td,
+.table > tfoot > tr.danger > td,
+.table > thead > tr.danger > th,
+.table > tbody > tr.danger > th,
+.table > tfoot > tr.danger > th {
+  background-color: #ff4136;
+}
+.table-hover > tbody > tr > td.danger:hover,
+.table-hover > tbody > tr > th.danger:hover,
+.table-hover > tbody > tr.danger:hover > td,
+.table-hover > tbody > tr.danger:hover > th {
+  background-color: #ff291c;
+}
+@media (max-width: 767px) {
+  .table-responsive {
+    width: 100%;
+    margin-bottom: 18.75px;
+    overflow-y: hidden;
+    overflow-x: scroll;
+    -ms-overflow-style: -ms-autohiding-scrollbar;
+    border: 1px solid #dddddd;
+    -webkit-overflow-scrolling: touch;
+  }
+  .table-responsive > .table {
+    margin-bottom: 0;
+  }
+  .table-responsive > .table > thead > tr > th,
+  .table-responsive > .table > tbody > tr > th,
+  .table-responsive > .table > tfoot > tr > th,
+  .table-responsive > .table > thead > tr > td,
+  .table-responsive > .table > tbody > tr > td,
+  .table-responsive > .table > tfoot > tr > td {
+    white-space: nowrap;
+  }
+  .table-responsive > .table-bordered {
+    border: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:first-child,
+  .table-responsive > .table-bordered > tbody > tr > th:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:first-child,
+  .table-responsive > .table-bordered > thead > tr > td:first-child,
+  .table-responsive > .table-bordered > tbody > tr > td:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:first-child {
+    border-left: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:last-child,
+  .table-responsive > .table-bordered > tbody > tr > th:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:last-child,
+  .table-responsive > .table-bordered > thead > tr > td:last-child,
+  .table-responsive > .table-bordered > tbody > tr > td:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:last-child {
+    border-right: 0;
+  }
+  .table-responsive > .table-bordered > tbody > tr:last-child > th,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > th,
+  .table-responsive > .table-bordered > tbody > tr:last-child > td,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > td {
+    border-bottom: 0;
+  }
+}
+fieldset {
+  padding: 0;
+  margin: 0;
+  border: 0;
+  min-width: 0;
+}
+legend {
+  display: block;
+  width: 100%;
+  padding: 0;
+  margin-bottom: 25px;
+  font-size: 27px;
+  line-height: inherit;
+  color: #333333;
+  border: 0;
+  border-bottom: 1px solid #e5e5e5;
+}
+label {
+  display: inline-block;
+  margin-bottom: 5px;
+  font-weight: bold;
+}
+input[type="search"] {
+  -webkit-box-sizing: border-box;
+  -moz-box-sizing: border-box;
+  box-sizing: border-box;
+}
+input[type="radio"],
+input[type="checkbox"] {
+  margin: 4px 0 0;
+  margin-top: 1px \9;
+  /* IE8-9 */
+  line-height: normal;
+}
+input[type="file"] {
+  display: block;
+}
+input[type="range"] {
+  display: block;
+  width: 100%;
+}
+select[multiple],
+select[size] {
+  height: auto;
+}
+input[type="file"]:focus,
+input[type="radio"]:focus,
+input[type="checkbox"]:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+output {
+  display: block;
+  padding-top: 7px;
+  font-size: 18px;
+  line-height: 1.42857143;
+  color: #555555;
+}
+.form-control {
+  display: block;
+  width: 100%;
+  height: 41px;
+  padding: 6px 12px;
+  font-size: 18px;
+  line-height: 1.42857143;
+  color: #555555;
+  background-color: #ffffff;
+  background-image: none;
+  border: 1px solid #e7e7e7;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+  transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+}
+.form-control:focus {
+  border-color: #66afe9;
+  outline: 0;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
+  box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);
+}
+.form-control::-moz-placeholder {
+  color: #999999;
+  opacity: 1;
+}
+.form-control:-ms-input-placeholder {
+  color: #999999;
+}
+.form-control::-webkit-input-placeholder {
+  color: #999999;
+}
+.form-control[disabled],
+.form-control[readonly],
+fieldset[disabled] .form-control {
+  cursor: not-allowed;
+  background-color: #eeeeee;
+  opacity: 1;
+}
+textarea.form-control {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: none;
+}
+input[type="date"] {
+  line-height: 41px;
+}
+.form-group {
+  margin-bottom: 15px;
+}
+.radio,
+.checkbox {
+  display: block;
+  min-height: 25px;
+  margin-top: 10px;
+  margin-bottom: 10px;
+  padding-left: 20px;
+}
+.radio label,
+.checkbox label {
+  display: inline;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio input[type="radio"],
+.radio-inline input[type="radio"],
+.checkbox input[type="checkbox"],
+.checkbox-inline input[type="checkbox"] {
+  float: left;
+  margin-left: -20px;
+}
+.radio + .radio,
+.checkbox + .checkbox {
+  margin-top: -5px;
+}
+.radio-inline,
+.checkbox-inline {
+  display: inline-block;
+  padding-left: 20px;
+  margin-bottom: 0;
+  vertical-align: middle;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio-inline + .radio-inline,
+.checkbox-inline + .checkbox-inline {
+  margin-top: 0;
+  margin-left: 10px;
+}
+input[type="radio"][disabled],
+input[type="checkbox"][disabled],
+.radio[disabled],
+.radio-inline[disabled],
+.checkbox[disabled],
+.checkbox-inline[disabled],
+fieldset[disabled] input[type="radio"],
+fieldset[disabled] input[type="checkbox"],
+fieldset[disabled] .radio,
+fieldset[disabled] .radio-inline,
+fieldset[disabled] .checkbox,
+fieldset[disabled] .checkbox-inline {
+  cursor: not-allowed;
+}
+.input-sm {
+  height: 36px;
+  padding: 5px 10px;
+  font-size: 16px;
+  line-height: 1.5;
+  border-radius: 2px;
+}
+select.input-sm {
+  height: 36px;
+  line-height: 36px;
+}
+textarea.input-sm,
+select[multiple].input-sm {
+  height: auto;
+}
+.input-lg {
+  height: 53px;
+  padding: 10px 16px;
+  font-size: 23px;
+  line-height: 1.33;
+  border-radius: 5px;
+}
+select.input-lg {
+  height: 53px;
+  line-height: 53px;
+}
+textarea.input-lg,
+select[multiple].input-lg {
+  height: auto;
+}
+.has-feedback {
+  position: relative;
+}
+.has-feedback .form-control {
+  padding-right: 51.25px;
+}
+.has-feedback .form-control-feedback {
+  position: absolute;
+  top: 30px;
+  right: 0;
+  display: block;
+  width: 41px;
+  height: 41px;
+  line-height: 41px;
+  text-align: center;
+}
+.has-success .help-block,
+.has-success .control-label,
+.has-success .radio,
+.has-success .checkbox,
+.has-success .radio-inline,
+.has-success .checkbox-inline {
+  color: #ffffff;
+}
+.has-success .form-control {
+  border-color: #ffffff;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-success .form-control:focus {
+  border-color: #e6e6e6;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ffffff;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ffffff;
+}
+.has-success .input-group-addon {
+  color: #ffffff;
+  border-color: #ffffff;
+  background-color: #28b62c;
+}
+.has-success .form-control-feedback {
+  color: #ffffff;
+}
+.has-warning .help-block,
+.has-warning .control-label,
+.has-warning .radio,
+.has-warning .checkbox,
+.has-warning .radio-inline,
+.has-warning .checkbox-inline {
+  color: #ffffff;
+}
+.has-warning .form-control {
+  border-color: #ffffff;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-warning .form-control:focus {
+  border-color: #e6e6e6;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ffffff;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ffffff;
+}
+.has-warning .input-group-addon {
+  color: #ffffff;
+  border-color: #ffffff;
+  background-color: #ff851b;
+}
+.has-warning .form-control-feedback {
+  color: #ffffff;
+}
+.has-error .help-block,
+.has-error .control-label,
+.has-error .radio,
+.has-error .checkbox,
+.has-error .radio-inline,
+.has-error .checkbox-inline {
+  color: #ffffff;
+}
+.has-error .form-control {
+  border-color: #ffffff;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);
+}
+.has-error .form-control:focus {
+  border-color: #e6e6e6;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ffffff;
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ffffff;
+}
+.has-error .input-group-addon {
+  color: #ffffff;
+  border-color: #ffffff;
+  background-color: #ff4136;
+}
+.has-error .form-control-feedback {
+  color: #ffffff;
+}
+.form-control-static {
+  margin-bottom: 0;
+}
+.help-block {
+  display: block;
+  margin-top: 5px;
+  margin-bottom: 10px;
+  color: #959595;
+}
+@media (min-width: 768px) {
+  .form-inline .form-group {
+    display: inline-block;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .form-control {
+    display: inline-block;
+    width: auto;
+    vertical-align: middle;
+  }
+  .form-inline .input-group > .form-control {
+    width: 100%;
+  }
+  .form-inline .control-label {
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio,
+  .form-inline .checkbox {
+    display: inline-block;
+    margin-top: 0;
+    margin-bottom: 0;
+    padding-left: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio input[type="radio"],
+  .form-inline .checkbox input[type="checkbox"] {
+    float: none;
+    margin-left: 0;
+  }
+  .form-inline .has-feedback .form-control-feedback {
+    top: 0;
+  }
+}
+.form-horizontal .control-label,
+.form-horizontal .radio,
+.form-horizontal .checkbox,
+.form-horizontal .radio-inline,
+.form-horizontal .checkbox-inline {
+  margin-top: 0;
+  margin-bottom: 0;
+  padding-top: 7px;
+}
+.form-horizontal .radio,
+.form-horizontal .checkbox {
+  min-height: 32px;
+}
+.form-horizontal .form-group {
+  margin-left: -15px;
+  margin-right: -15px;
+}
+.form-horizontal .form-control-static {
+  padding-top: 7px;
+}
+@media (min-width: 768px) {
+  .form-horizontal .control-label {
+    text-align: right;
+  }
+}
+.form-horizontal .has-feedback .form-control-feedback {
+  top: 0;
+  right: 15px;
+}
+.btn {
+  display: inline-block;
+  margin-bottom: 0;
+  font-weight: normal;
+  text-align: center;
+  vertical-align: middle;
+  cursor: pointer;
+  background-image: none;
+  border: 1px solid transparent;
+  white-space: nowrap;
+  padding: 6px 12px;
+  font-size: 18px;
+  line-height: 1.42857143;
+  border-radius: 4px;
+  -webkit-user-select: none;
+  -moz-user-select: none;
+  -ms-user-select: none;
+  user-select: none;
+}
+.btn:focus,
+.btn:active:focus,
+.btn.active:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+.btn:hover,
+.btn:focus {
+  color: #555555;
+  text-decoration: none;
+}
+.btn:active,
+.btn.active {
+  outline: 0;
+  background-image: none;
+  -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);
+  box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);
+}
+.btn.disabled,
+.btn[disabled],
+fieldset[disabled] .btn {
+  cursor: not-allowed;
+  pointer-events: none;
+  opacity: 0.65;
+  filter: alpha(opacity=65);
+  -webkit-box-shadow: none;
+  box-shadow: none;
+}
+.btn-default {
+  color: #555555;
+  background-color: #eeeeee;
+  border-color: #e2e2e2;
+}
+.btn-default:hover,
+.btn-default:focus,
+.btn-default:active,
+.btn-default.active,
+.open .dropdown-toggle.btn-default {
+  color: #555555;
+  background-color: #dadada;
+  border-color: #c3c3c3;
+}
+.btn-default:active,
+.btn-default.active,
+.open .dropdown-toggle.btn-default {
+  background-image: none;
+}
+.btn-default.disabled,
+.btn-default[disabled],
+fieldset[disabled] .btn-default,
+.btn-default.disabled:hover,
+.btn-default[disabled]:hover,
+fieldset[disabled] .btn-default:hover,
+.btn-default.disabled:focus,
+.btn-default[disabled]:focus,
+fieldset[disabled] .btn-default:focus,
+.btn-default.disabled:active,
+.btn-default[disabled]:active,
+fieldset[disabled] .btn-default:active,
+.btn-default.disabled.active,
+.btn-default[disabled].active,
+fieldset[disabled] .btn-default.active {
+  background-color: #eeeeee;
+  border-color: #e2e2e2;
+}
+.btn-default .badge {
+  color: #eeeeee;
+  background-color: #555555;
+}
+.btn-primary {
+  color: #ffffff;
+  background-color: #158cba;
+  border-color: #127ba3;
+}
+.btn-primary:hover,
+.btn-primary:focus,
+.btn-primary:active,
+.btn-primary.active,
+.open .dropdown-toggle.btn-primary {
+  color: #ffffff;
+  background-color: #117095;
+  border-color: #0c516c;
+}
+.btn-primary:active,
+.btn-primary.active,
+.open .dropdown-toggle.btn-primary {
+  background-image: none;
+}
+.btn-primary.disabled,
+.btn-primary[disabled],
+fieldset[disabled] .btn-primary,
+.btn-primary.disabled:hover,
+.btn-primary[disabled]:hover,
+fieldset[disabled] .btn-primary:hover,
+.btn-primary.disabled:focus,
+.btn-primary[disabled]:focus,
+fieldset[disabled] .btn-primary:focus,
+.btn-primary.disabled:active,
+.btn-primary[disabled]:active,
+fieldset[disabled] .btn-primary:active,
+.btn-primary.disabled.active,
+.btn-primary[disabled].active,
+fieldset[disabled] .btn-primary.active {
+  background-color: #158cba;
+  border-color: #127ba3;
+}
+.btn-primary .badge {
+  color: #158cba;
+  background-color: #ffffff;
+}
+.btn-success {
+  color: #ffffff;
+  background-color: #28b62c;
+  border-color: #23a127;
+}
+.btn-success:hover,
+.btn-success:focus,
+.btn-success:active,
+.btn-success.active,
+.open .dropdown-toggle.btn-success {
+  color: #ffffff;
+  background-color: #219524;
+  border-color: #186f1b;
+}
+.btn-success:active,
+.btn-success.active,
+.open .dropdown-toggle.btn-success {
+  background-image: none;
+}
+.btn-success.disabled,
+.btn-success[disabled],
+fieldset[disabled] .btn-success,
+.btn-success.disabled:hover,
+.btn-success[disabled]:hover,
+fieldset[disabled] .btn-success:hover,
+.btn-success.disabled:focus,
+.btn-success[disabled]:focus,
+fieldset[disabled] .btn-success:focus,
+.btn-success.disabled:active,
+.btn-success[disabled]:active,
+fieldset[disabled] .btn-success:active,
+.btn-success.disabled.active,
+.btn-success[disabled].active,
+fieldset[disabled] .btn-success.active {
+  background-color: #28b62c;
+  border-color: #23a127;
+}
+.btn-success .badge {
+  color: #28b62c;
+  background-color: #ffffff;
+}
+.btn-info {
+  color: #ffffff;
+  background-color: #75caeb;
+  border-color: #5fc1e8;
+}
+.btn-info:hover,
+.btn-info:focus,
+.btn-info:active,
+.btn-info.active,
+.open .dropdown-toggle.btn-info {
+  color: #ffffff;
+  background-color: #51bce6;
+  border-color: #29ade0;
+}
+.btn-info:active,
+.btn-info.active,
+.open .dropdown-toggle.btn-info {
+  background-image: none;
+}
+.btn-info.disabled,
+.btn-info[disabled],
+fieldset[disabled] .btn-info,
+.btn-info.disabled:hover,
+.btn-info[disabled]:hover,
+fieldset[disabled] .btn-info:hover,
+.btn-info.disabled:focus,
+.btn-info[disabled]:focus,
+fieldset[disabled] .btn-info:focus,
+.btn-info.disabled:active,
+.btn-info[disabled]:active,
+fieldset[disabled] .btn-info:active,
+.btn-info.disabled.active,
+.btn-info[disabled].active,
+fieldset[disabled] .btn-info.active {
+  background-color: #75caeb;
+  border-color: #5fc1e8;
+}
+.btn-info .badge {
+  color: #75caeb;
+  background-color: #ffffff;
+}
+.btn-warning {
+  color: #ffffff;
+  background-color: #ff851b;
+  border-color: #ff7701;
+}
+.btn-warning:hover,
+.btn-warning:focus,
+.btn-warning:active,
+.btn-warning.active,
+.open .dropdown-toggle.btn-warning {
+  color: #ffffff;
+  background-color: #f17000;
+  border-color: #c35b00;
+}
+.btn-warning:active,
+.btn-warning.active,
+.open .dropdown-toggle.btn-warning {
+  background-image: none;
+}
+.btn-warning.disabled,
+.btn-warning[disabled],
+fieldset[disabled] .btn-warning,
+.btn-warning.disabled:hover,
+.btn-warning[disabled]:hover,
+fieldset[disabled] .btn-warning:hover,
+.btn-warning.disabled:focus,
+.btn-warning[disabled]:focus,
+fieldset[disabled] .btn-warning:focus,
+.btn-warning.disabled:active,
+.btn-warning[disabled]:active,
+fieldset[disabled] .btn-warning:active,
+.btn-warning.disabled.active,
+.btn-warning[disabled].active,
+fieldset[disabled] .btn-warning.active {
+  background-color: #ff851b;
+  border-color: #ff7701;
+}
+.btn-warning .badge {
+  color: #ff851b;
+  background-color: #ffffff;
+}
+.btn-danger {
+  color: #ffffff;
+  background-color: #ff4136;
+  border-color: #ff291c;
+}
+.btn-danger:hover,
+.btn-danger:focus,
+.btn-danger:active,
+.btn-danger.active,
+.open .dropdown-toggle.btn-danger {
+  color: #ffffff;
+  background-color: #ff1a0d;
+  border-color: #de0c00;
+}
+.btn-danger:active,
+.btn-danger.active,
+.open .dropdown-toggle.btn-danger {
+  background-image: none;
+}
+.btn-danger.disabled,
+.btn-danger[disabled],
+fieldset[disabled] .btn-danger,
+.btn-danger.disabled:hover,
+.btn-danger[disabled]:hover,
+fieldset[disabled] .btn-danger:hover,
+.btn-danger.disabled:focus,
+.btn-danger[disabled]:focus,
+fieldset[disabled] .btn-danger:focus,
+.btn-danger.disabled:active,
+.btn-danger[disabled]:active,
+fieldset[disabled] .btn-danger:active,
+.btn-danger.disabled.active,
+.btn-danger[disabled].active,
+fieldset[disabled] .btn-danger.active {
+  background-color: #ff4136;
+  border-color: #ff291c;
+}
+.btn-danger .badge {
+  color: #ff4136;
+  background-color: #ffffff;
+}
+.btn-link {
+  color: #158cba;
+  font-weight: normal;
+  cursor: pointer;
+  border-radius: 0;
+}
+.btn-link,
+.btn-link:active,
+.btn-link[disabled],
+fieldset[disabled] .btn-link {
+  background-color: transparent;
+  -webkit-box-shadow: none;
+  box-shadow: none;
+}
+.btn-link,
+.btn-link:hover,
+.btn-link:focus,
+.btn-link:active {
+  border-color: transparent;
+}
+.btn-link:hover,
+.btn-link:focus {
+  color: #158cba;
+  text-decoration: underline;
+  background-color: transparent;
+}
+.btn-link[disabled]:hover,
+fieldset[disabled] .btn-link:hover,
+.btn-link[disabled]:focus,
+fieldset[disabled] .btn-link:focus {
+  color: #999999;
+  text-decoration: none;
+}
+.btn-lg,
+.btn-group-lg > .btn {
+  padding: 10px 16px;
+  font-size: 23px;
+  line-height: 1.33;
+  border-radius: 5px;
+}
+.btn-sm,
+.btn-group-sm > .btn {
+  padding: 5px 10px;
+  font-size: 16px;
+  line-height: 1.5;
+  border-radius: 2px;
+}
+.btn-xs,
+.btn-group-xs > .btn {
+  padding: 1px 5px;
+  font-size: 16px;
+  line-height: 1.5;
+  border-radius: 2px;
+}
+.btn-block {
+  display: block;
+  width: 100%;
+  padding-left: 0;
+  padding-right: 0;
+}
+.btn-block + .btn-block {
+  margin-top: 5px;
+}
+input[type="submit"].btn-block,
+input[type="reset"].btn-block,
+input[type="button"].btn-block {
+  width: 100%;
+}
+.fade {
+  opacity: 0;
+  -webkit-transition: opacity 0.15s linear;
+  transition: opacity 0.15s linear;
+}
+.fade.in {
+  opacity: 1;
+}
+.collapse {
+  display: none;
+}
+.collapse.in {
+  display: block;
+}
+.collapsing {
+  position: relative;
+  height: 0;
+  overflow: hidden;
+  -webkit-transition: height 0.35s ease;
+  transition: height 0.35s ease;
+}
+@font-face {
+  font-family: 'Glyphicons Halflings';
+  src: url('../fonts/glyphicons-halflings-regular.eot');
+  src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');
+}
+.glyphicon {
+  position: relative;
+  top: 1px;
+  display: inline-block;
+  font-family: 'Glyphicons Halflings';
+  font-style: normal;
+  font-weight: normal;
+  line-height: 1;
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+}
+.glyphicon-asterisk:before {
+  content: "\2a";
+}
+.glyphicon-plus:before {
+  content: "\2b";
+}
+.glyphicon-euro:before {
+  content: "\20ac";
+}
+.glyphicon-minus:before {
+  content: "\2212";
+}
+.glyphicon-cloud:before {
+  content: "\2601";
+}
+.glyphicon-envelope:before {
+  content: "\2709";
+}
+.glyphicon-pencil:before {
+  content: "\270f";
+}
+.glyphicon-glass:before {
+  content: "\e001";
+}
+.glyphicon-music:before {
+  content: "\e002";
+}
+.glyphicon-search:before {
+  content: "\e003";
+}
+.glyphicon-heart:before {
+  content: "\e005";
+}
+.glyphicon-star:before {
+  content: "\e006";
+}
+.glyphicon-star-empty:before {
+  content: "\e007";
+}
+.glyphicon-user:before {
+  content: "\e008";
+}
+.glyphicon-film:before {
+  content: "\e009";
+}
+.glyphicon-th-large:before {
+  content: "\e010";
+}
+.glyphicon-th:before {
+  content: "\e011";
+}
+.glyphicon-th-list:before {
+  content: "\e012";
+}
+.glyphicon-ok:before {
+  content: "\e013";
+}
+.glyphicon-remove:before {
+  content: "\e014";
+}
+.glyphicon-zoom-in:before {
+  content: "\e015";
+}
+.glyphicon-zoom-out:before {
+  content: "\e016";
+}
+.glyphicon-off:before {
+  content: "\e017";
+}
+.glyphicon-signal:before {
+  content: "\e018";
+}
+.glyphicon-cog:before {
+  content: "\e019";
+}
+.glyphicon-trash:before {
+  content: "\e020";
+}
+.glyphicon-home:before {
+  content: "\e021";
+}
+.glyphicon-file:before {
+  content: "\e022";
+}
+.glyphicon-time:before {
+  content: "\e023";
+}
+.glyphicon-road:before {
+  content: "\e024";
+}
+.glyphicon-download-alt:before {
+  content: "\e025";
+}
+.glyphicon-download:before {
+  content: "\e026";
+}
+.glyphicon-upload:before {
+  content: "\e027";
+}
+.glyphicon-inbox:before {
+  content: "\e028";
+}
+.glyphicon-play-circle:before {
+  content: "\e029";
+}
+.glyphicon-repeat:before {
+  content: "\e030";
+}
+.glyphicon-refresh:before {
+  content: "\e031";
+}
+.glyphicon-list-alt:before {
+  content: "\e032";
+}
+.glyphicon-lock:before {
+  content: "\e033";
+}
+.glyphicon-flag:before {
+  content: "\e034";
+}
+.glyphicon-headphones:before {
+  content: "\e035";
+}
+.glyphicon-volume-off:before {
+  content: "\e036";
+}
+.glyphicon-volume-down:before {
+  content: "\e037";
+}
+.glyphicon-volume-up:before {
+  content: "\e038";
+}
+.glyphicon-qrcode:before {
+  content: "\e039";
+}
+.glyphicon-barcode:before {
+  content: "\e040";
+}
+.glyphicon-tag:before {
+  content: "\e041";
+}
+.glyphicon-tags:before {
+  content: "\e042";
+}
+.glyphicon-book:before {
+  content: "\e043";
+}
+.glyphicon-bookmark:before {
+  content: "\e044";
+}
+.glyphicon-print:before {
+  content: "\e045";
+}
+.glyphicon-camera:before {
+  content: "\e046";
+}
+.glyphicon-font:before {
+  content: "\e047";
+}
+.glyphicon-bold:before {
+  content: "\e048";
+}
+.glyphicon-italic:before {
+  content: "\e049";
+}
+.glyphicon-text-height:before {
+  content: "\e050";
+}
+.glyphicon-text-width:before {
+  content: "\e051";
+}
+.glyphicon-align-left:before {
+  content: "\e052";
+}
+.glyphicon-align-center:before {
+  content: "\e053";
+}
+.glyphicon-align-right:before {
+  content: "\e054";
+}
+.glyphicon-align-justify:before {
+  content: "\e055";
+}
+.glyphicon-list:before {
+  content: "\e056";
+}
+.glyphicon-indent-left:before {
+  content: "\e057";
+}
+.glyphicon-indent-right:before {
+  content: "\e058";
+}
+.glyphicon-facetime-video:before {
+  content: "\e059";
+}
+.glyphicon-picture:before {
+  content: "\e060";
+}
+.glyphicon-map-marker:before {
+  content: "\e062";
+}
+.glyphicon-adjust:before {
+  content: "\e063";
+}
+.glyphicon-tint:before {
+  content: "\e064";
+}
+.glyphicon-edit:before {
+  content: "\e065";
+}
+.glyphicon-share:before {
+  content: "\e066";
+}
+.glyphicon-check:before {
+  content: "\e067";
+}
+.glyphicon-move:before {
+  content: "\e068";
+}
+.glyphicon-step-backward:before {
+  content: "\e069";
+}
+.glyphicon-fast-backward:before {
+  content: "\e070";
+}
+.glyphicon-backward:before {
+  content: "\e071";
+}
+.glyphicon-play:before {
+  content: "\e072";
+}
+.glyphicon-pause:before {
+  content: "\e073";
+}
+.glyphicon-stop:before {
+  content: "\e074";
+}
+.glyphicon-forward:before {
+  content: "\e075";
+}
+.glyphicon-fast-forward:before {
+  content: "\e076";
+}
+.glyphicon-step-forward:before {
+  content: "\e077";
+}
+.glyphicon-eject:before {
+  content: "\e078";
+}
+.glyphicon-chevron-left:before {
+  content: "\e079";
+}
+.glyphicon-chevron-right:before {
+  content: "\e080";
+}
+.glyphicon-plus-sign:before {
+  content: "\e081";
+}
+.glyphicon-minus-sign:before {
+  content: "\e082";
+}
+.glyphicon-remove-sign:before {
+  content: "\e083";
+}
+.glyphicon-ok-sign:before {
+  content: "\e084";
+}
+.glyphicon-question-sign:before {
+  content: "\e085";
+}
+.glyphicon-info-sign:before {
+  content: "\e086";
+}
+.glyphicon-screenshot:before {
+  content: "\e087";
+}
+.glyphicon-remove-circle:before {
+  content: "\e088";
+}
+.glyphicon-ok-circle:before {
+  content: "\e089";
+}
+.glyphicon-ban-circle:before {
+  content: "\e090";
+}
+.glyphicon-arrow-left:before {
+  content: "\e091";
+}
+.glyphicon-arrow-right:before {
+  content: "\e092";
+}
+.glyphicon-arrow-up:before {
+  content: "\e093";
+}
+.glyphicon-arrow-down:before {
+  content: "\e094";
+}
+.glyphicon-share-alt:before {
+  content: "\e095";
+}
+.glyphicon-resize-full:before {
+  content: "\e096";
+}
+.glyphicon-resize-small:before {
+  content: "\e097";
+}
+.glyphicon-exclamation-sign:before {
+  content: "\e101";
+}
+.glyphicon-gift:before {
+  content: "\e102";
+}
+.glyphicon-leaf:before {
+  content: "\e103";
+}
+.glyphicon-fire:before {
+  content: "\e104";
+}
+.glyphicon-eye-open:before {
+  content: "\e105";
+}
+.glyphicon-eye-close:before {
+  content: "\e106";
+}
+.glyphicon-warning-sign:before {
+  content: "\e107";
+}
+.glyphicon-plane:before {
+  content: "\e108";
+}
+.glyphicon-calendar:before {
+  content: "\e109";
+}
+.glyphicon-random:before {
+  content: "\e110";
+}
+.glyphicon-comment:before {
+  content: "\e111";
+}
+.glyphicon-magnet:before {
+  content: "\e112";
+}
+.glyphicon-chevron-up:before {
+  content: "\e113";
+}
+.glyphicon-chevron-down:before {
+  content: "\e114";
+}
+.glyphicon-retweet:before {
+  content: "\e115";
+}
+.glyphicon-shopping-cart:before {
+  content: "\e116";
+}
+.glyphicon-folder-close:before {
+  content: "\e117";
+}
+.glyphicon-folder-open:before {
+  content: "\e118";
+}
+.glyphicon-resize-vertical:before {
+  content: "\e119";
+}
+.glyphicon-resize-horizontal:before {
+  content: "\e120";
+}
+.glyphicon-hdd:before {
+  content: "\e121";
+}
+.glyphicon-bullhorn:before {
+  content: "\e122";
+}
+.glyphicon-bell:before {
+  content: "\e123";
+}
+.glyphicon-certificate:before {
+  content: "\e124";
+}
+.glyphicon-thumbs-up:before {
+  content: "\e125";
+}
+.glyphicon-thumbs-down:before {
+  content: "\e126";
+}
+.glyphicon-hand-right:before {
+  content: "\e127";
+}
+.glyphicon-hand-left:before {
+  content: "\e128";
+}
+.glyphicon-hand-up:before {
+  content: "\e129";
+}
+.glyphicon-hand-down:before {
+  content: "\e130";
+}
+.glyphicon-circle-arrow-right:before {
+  content: "\e131";
+}
+.glyphicon-circle-arrow-left:before {
+  content: "\e132";
+}
+.glyphicon-circle-arrow-up:before {
+  content: "\e133";
+}
+.glyphicon-circle-arrow-down:before {
+  content: "\e134";
+}
+.glyphicon-globe:before {
+  content: "\e135";
+}
+.glyphicon-wrench:before {
+  content: "\e136";
+}
+.glyphicon-tasks:before {
+  content: "\e137";
+}
+.glyphicon-filter:before {
+  content: "\e138";
+}
+.glyphicon-briefcase:before {
+  content: "\e139";
+}
+.glyphicon-fullscreen:before {
+  content: "\e140";
+}
+.glyphicon-dashboard:before {
+  content: "\e141";
+}
+.glyphicon-paperclip:before {
+  content: "\e142";
+}
+.glyphicon-heart-empty:before {
+  content: "\e143";
+}
+.glyphicon-link:before {
+  content: "\e144";
+}
+.glyphicon-phone:before {
+  content: "\e145";
+}
+.glyphicon-pushpin:before {
+  content: "\e146";
+}
+.glyphicon-usd:before {
+  content: "\e148";
+}
+.glyphicon-gbp:before {
+  content: "\e149";
+}
+.glyphicon-sort:before {
+  content: "\e150";
+}
+.glyphicon-sort-by-alphabet:before {
+  content: "\e151";
+}
+.glyphicon-sort-by-alphabet-alt:before {
+  content: "\e152";
+}
+.glyphicon-sort-by-order:before {
+  content: "\e153";
+}
+.glyphicon-sort-by-order-alt:before {
+  content: "\e154";
+}
+.glyphicon-sort-by-attributes:before {
+  content: "\e155";
+}
+.glyphicon-sort-by-attributes-alt:before {
+  content: "\e156";
+}
+.glyphicon-unchecked:before {
+  content: "\e157";
+}
+.glyphicon-expand:before {
+  content: "\e158";
+}
+.glyphicon-collapse-down:before {
+  content: "\e159";
+}
+.glyphicon-collapse-up:before {
+  content: "\e160";
+}
+.glyphicon-log-in:before {
+  content: "\e161";
+}
+.glyphicon-flash:before {
+  content: "\e162";
+}
+.glyphicon-log-out:before {
+  content: "\e163";
+}
+.glyphicon-new-window:before {
+  content: "\e164";
+}
+.glyphicon-record:before {
+  content: "\e165";
+}
+.glyphicon-save:before {
+  content: "\e166";
+}
+.glyphicon-open:before {
+  content: "\e167";
+}
+.glyphicon-saved:before {
+  content: "\e168";
+}
+.glyphicon-import:before {
+  content: "\e169";
+}
+.glyphicon-export:before {
+  content: "\e170";
+}
+.glyphicon-send:before {
+  content: "\e171";
+}
+.glyphicon-floppy-disk:before {
+  content: "\e172";
+}
+.glyphicon-floppy-saved:before {
+  content: "\e173";
+}
+.glyphicon-floppy-remove:before {
+  content: "\e174";
+}
+.glyphicon-floppy-save:before {
+  content: "\e175";
+}
+.glyphicon-floppy-open:before {
+  content: "\e176";
+}
+.glyphicon-credit-card:before {
+  content: "\e177";
+}
+.glyphicon-transfer:before {
+  content: "\e178";
+}
+.glyphicon-cutlery:before {
+  content: "\e179";
+}
+.glyphicon-header:before {
+  content: "\e180";
+}
+.glyphicon-compressed:before {
+  content: "\e181";
+}
+.glyphicon-earphone:before {
+  content: "\e182";
+}
+.glyphicon-phone-alt:before {
+  content: "\e183";
+}
+.glyphicon-tower:before {
+  content: "\e184";
+}
+.glyphicon-stats:before {
+  content: "\e185";
+}
+.glyphicon-sd-video:before {
+  content: "\e186";
+}
+.glyphicon-hd-video:before {
+  content: "\e187";
+}
+.glyphicon-subtitles:before {
+  content: "\e188";
+}
+.glyphicon-sound-stereo:before {
+  content: "\e189";
+}
+.glyphicon-sound-dolby:before {
+  content: "\e190";
+}
+.glyphicon-sound-5-1:before {
+  content: "\e191";
+}
+.glyphicon-sound-6-1:before {
+  content: "\e192";
+}
+.glyphicon-sound-7-1:before {
+  content: "\e193";
+}
+.glyphicon-copyright-mark:before {
+  content: "\e194";
+}
+.glyphicon-registration-mark:before {
+  content: "\e195";
+}
+.glyphicon-cloud-download:before {
+  content: "\e197";
+}
+.glyphicon-cloud-upload:before {
+  content: "\e198";
+}
+.glyphicon-tree-conifer:before {
+  content: "\e199";
+}
+.glyphicon-tree-deciduous:before {
+  content: "\e200";
+}
+.caret {
+  display: inline-block;
+  width: 0;
+  height: 0;
+  margin-left: 2px;
+  vertical-align: middle;
+  border-top: 4px solid;
+  border-right: 4px solid transparent;
+  border-left: 4px solid transparent;
+}
+.dropdown {
+  position: relative;
+}
+.dropdown-toggle:focus {
+  outline: 0;
+}
+.dropdown-menu {
+  position: absolute;
+  top: 100%;
+  left: 0;
+  z-index: 1000;
+  display: none;
+  float: left;
+  min-width: 160px;
+  padding: 5px 0;
+  margin: 2px 0 0;
+  list-style: none;
+  font-size: 18px;
+  background-color: #ffffff;
+  border: 1px solid #cccccc;
+  border: 1px solid #e7e7e7;
+  border-radius: 4px;
+  -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175);
+  box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175);
+  background-clip: padding-box;
+}
+.dropdown-menu.pull-right {
+  right: 0;
+  left: auto;
+}
+.dropdown-menu .divider {
+  height: 1px;
+  margin: 11.5px 0;
+  overflow: hidden;
+  background-color: #eeeeee;
+}
+.dropdown-menu > li > a {
+  display: block;
+  padding: 3px 20px;
+  clear: both;
+  font-weight: normal;
+  line-height: 1.42857143;
+  color: #999999;
+  white-space: nowrap;
+}
+.dropdown-menu > li > a:hover,
+.dropdown-menu > li > a:focus {
+  text-decoration: none;
+  color: #333333;
+  background-color: transparent;
+}
+.dropdown-menu > .active > a,
+.dropdown-menu > .active > a:hover,
+.dropdown-menu > .active > a:focus {
+  color: #ffffff;
+  text-decoration: none;
+  outline: 0;
+  background-color: #158cba;
+}
+.dropdown-menu > .disabled > a,
+.dropdown-menu > .disabled > a:hover,
+.dropdown-menu > .disabled > a:focus {
+  color: #eeeeee;
+}
+.dropdown-menu > .disabled > a:hover,
+.dropdown-menu > .disabled > a:focus {
+  text-decoration: none;
+  background-color: transparent;
+  background-image: none;
+  filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);
+  cursor: not-allowed;
+}
+.open > .dropdown-menu {
+  display: block;
+}
+.open > a {
+  outline: 0;
+}
+.dropdown-menu-right {
+  left: auto;
+  right: 0;
+}
+.dropdown-menu-left {
+  left: 0;
+  right: auto;
+}
+.dropdown-header {
+  display: block;
+  padding: 3px 20px;
+  font-size: 16px;
+  line-height: 1.42857143;
+  color: #999999;
+}
+.dropdown-backdrop {
+  position: fixed;
+  left: 0;
+  right: 0;
+  bottom: 0;
+  top: 0;
+  z-index: 990;
+}
+.pull-right > .dropdown-menu {
+  right: 0;
+  left: auto;
+}
+.dropup .caret,
+.navbar-fixed-bottom .dropdown .caret {
+  border-top: 0;
+  border-bottom: 4px solid;
+  content: "";
+}
+.dropup .dropdown-menu,
+.navbar-fixed-bottom .dropdown .dropdown-menu {
+  top: auto;
+  bottom: 100%;
+  margin-bottom: 1px;
+}
+@media (min-width: 768px) {
+  .navbar-right .dropdown-menu {
+    left: auto;
+    right: 0;
+  }
+  .navbar-right .dropdown-menu-left {
+    left: 0;
+    right: auto;
+  }
+}
+.btn-group,
+.btn-group-vertical {
+  position: relative;
+  display: inline-block;
+  vertical-align: middle;
+}
+.btn-group > .btn,
+.btn-group-vertical > .btn {
+  position: relative;
+  float: left;
+}
+.btn-group > .btn:hover,
+.btn-group-vertical > .btn:hover,
+.btn-group > .btn:focus,
+.btn-group-vertical > .btn:focus,
+.btn-group > .btn:active,
+.btn-group-vertical > .btn:active,
+.btn-group > .btn.active,
+.btn-group-vertical > .btn.active {
+  z-index: 2;
+}
+.btn-group > .btn:focus,
+.btn-group-vertical > .btn:focus {
+  outline: none;
+}
+.btn-group .btn + .btn,
+.btn-group .btn + .btn-group,
+.btn-group .btn-group + .btn,
+.btn-group .btn-group + .btn-group {
+  margin-left: -1px;
+}
+.btn-toolbar {
+  margin-left: -5px;
+}
+.btn-toolbar .btn-group,
+.btn-toolbar .input-group {
+  float: left;
+}
+.btn-toolbar > .btn,
+.btn-toolbar > .btn-group,
+.btn-toolbar > .input-group {
+  margin-left: 5px;
+}
+.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) {
+  border-radius: 0;
+}
+.btn-group > .btn:first-child {
+  margin-left: 0;
+}
+.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) {
+  border-bottom-right-radius: 0;
+  border-top-right-radius: 0;
+}
+.btn-group > .btn:last-child:not(:first-child),
+.btn-group > .dropdown-toggle:not(:first-child) {
+  border-bottom-left-radius: 0;
+  border-top-left-radius: 0;
+}
+.btn-group > .btn-group {
+  float: left;
+}
+.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn {
+  border-radius: 0;
+}
+.btn-group > .btn-group:first-child > .btn:last-child,
+.btn-group > .btn-group:first-child > .dropdown-toggle {
+  border-bottom-right-radius: 0;
+  border-top-right-radius: 0;
+}
+.btn-group > .btn-group:last-child > .btn:first-child {
+  border-bottom-left-radius: 0;
+  border-top-left-radius: 0;
+}
+.btn-group .dropdown-toggle:active,
+.btn-group.open .dropdown-toggle {
+  outline: 0;
+}
+.btn-group > .btn + .dropdown-toggle {
+  padding-left: 8px;
+  padding-right: 8px;
+}
+.btn-group > .btn-lg + .dropdown-toggle {
+  padding-left: 12px;
+  padding-right: 12px;
+}
+.btn-group.open .dropdown-toggle {
+  -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);
+  box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);
+}
+.btn-group.open .dropdown-toggle.btn-link {
+  -webkit-box-shadow: none;
+  box-shadow: none;
+}
+.btn .caret {
+  margin-left: 0;
+}
+.btn-lg .caret {
+  border-width: 5px 5px 0;
+  border-bottom-width: 0;
+}
+.dropup .btn-lg .caret {
+  border-width: 0 5px 5px;
+}
+.btn-group-vertical > .btn,
+.btn-group-vertical > .btn-group,
+.btn-group-vertical > .btn-group > .btn {
+  display: block;
+  float: none;
+  width: 100%;
+  max-width: 100%;
+}
+.btn-group-vertical > .btn-group > .btn {
+  float: none;
+}
+.btn-group-vertical > .btn + .btn,
+.btn-group-vertical > .btn + .btn-group,
+.btn-group-vertical > .btn-group + .btn,
+.btn-group-vertical > .btn-group + .btn-group {
+  margin-top: -1px;
+  margin-left: 0;
+}
+.btn-group-vertical > .btn:not(:first-child):not(:last-child) {
+  border-radius: 0;
+}
+.btn-group-vertical > .btn:first-child:not(:last-child) {
+  border-top-right-radius: 4px;
+  border-bottom-right-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.btn-group-vertical > .btn:last-child:not(:first-child) {
+  border-bottom-left-radius: 4px;
+  border-top-right-radius: 0;
+  border-top-left-radius: 0;
+}
+.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn {
+  border-radius: 0;
+}
+.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child,
+.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle {
+  border-bottom-right-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child {
+  border-top-right-radius: 0;
+  border-top-left-radius: 0;
+}
+.btn-group-justified {
+  display: table;
+  width: 100%;
+  table-layout: fixed;
+  border-collapse: separate;
+}
+.btn-group-justified > .btn,
+.btn-group-justified > .btn-group {
+  float: none;
+  display: table-cell;
+  width: 1%;
+}
+.btn-group-justified > .btn-group .btn {
+  width: 100%;
+}
+[data-toggle="buttons"] > .btn > input[type="radio"],
+[data-toggle="buttons"] > .btn > input[type="checkbox"] {
+  display: none;
+}
+.input-group {
+  position: relative;
+  display: table;
+  border-collapse: separate;
+}
+.input-group[class*="col-"] {
+  float: none;
+  padding-left: 0;
+  padding-right: 0;
+}
+.input-group .form-control {
+  position: relative;
+  z-index: 2;
+  float: left;
+  width: 100%;
+  margin-bottom: 0;
+}
+.input-group-lg > .form-control,
+.input-group-lg > .input-group-addon,
+.input-group-lg > .input-group-btn > .btn {
+  height: 53px;
+  padding: 10px 16px;
+  font-size: 23px;
+  line-height: 1.33;
+  border-radius: 5px;
+}
+select.input-group-lg > .form-control,
+select.input-group-lg > .input-group-addon,
+select.input-group-lg > .input-group-btn > .btn {
+  height: 53px;
+  line-height: 53px;
+}
+textarea.input-group-lg > .form-control,
+textarea.input-group-lg > .input-group-addon,
+textarea.input-group-lg > .input-group-btn > .btn,
+select[multiple].input-group-lg > .form-control,
+select[multiple].input-group-lg > .input-group-addon,
+select[multiple].input-group-lg > .input-group-btn > .btn {
+  height: auto;
+}
+.input-group-sm > .form-control,
+.input-group-sm > .input-group-addon,
+.input-group-sm > .input-group-btn > .btn {
+  height: 36px;
+  padding: 5px 10px;
+  font-size: 16px;
+  line-height: 1.5;
+  border-radius: 2px;
+}
+select.input-group-sm > .form-control,
+select.input-group-sm > .input-group-addon,
+select.input-group-sm > .input-group-btn > .btn {
+  height: 36px;
+  line-height: 36px;
+}
+textarea.input-group-sm > .form-control,
+textarea.input-group-sm > .input-group-addon,
+textarea.input-group-sm > .input-group-btn > .btn,
+select[multiple].input-group-sm > .form-control,
+select[multiple].input-group-sm > .input-group-addon,
+select[multiple].input-group-sm > .input-group-btn > .btn {
+  height: auto;
+}
+.input-group-addon,
+.input-group-btn,
+.input-group .form-control {
+  display: table-cell;
+}
+.input-group-addon:not(:first-child):not(:last-child),
+.input-group-btn:not(:first-child):not(:last-child),
+.input-group .form-control:not(:first-child):not(:last-child) {
+  border-radius: 0;
+}
+.input-group-addon,
+.input-group-btn {
+  width: 1%;
+  white-space: nowrap;
+  vertical-align: middle;
+}
+.input-group-addon {
+  padding: 6px 12px;
+  font-size: 18px;
+  font-weight: normal;
+  line-height: 1;
+  color: #555555;
+  text-align: center;
+  background-color: #eeeeee;
+  border: 1px solid #e7e7e7;
+  border-radius: 4px;
+}
+.input-group-addon.input-sm {
+  padding: 5px 10px;
+  font-size: 16px;
+  border-radius: 2px;
+}
+.input-group-addon.input-lg {
+  padding: 10px 16px;
+  font-size: 23px;
+  border-radius: 5px;
+}
+.input-group-addon input[type="radio"],
+.input-group-addon input[type="checkbox"] {
+  margin-top: 0;
+}
+.input-group .form-control:first-child,
+.input-group-addon:first-child,
+.input-group-btn:first-child > .btn,
+.input-group-btn:first-child > .btn-group > .btn,
+.input-group-btn:first-child > .dropdown-toggle,
+.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle),
+.input-group-btn:last-child > .btn-group:not(:last-child) > .btn {
+  border-bottom-right-radius: 0;
+  border-top-right-radius: 0;
+}
+.input-group-addon:first-child {
+  border-right: 0;
+}
+.input-group .form-control:last-child,
+.input-group-addon:last-child,
+.input-group-btn:last-child > .btn,
+.input-group-btn:last-child > .btn-group > .btn,
+.input-group-btn:last-child > .dropdown-toggle,
+.input-group-btn:first-child > .btn:not(:first-child),
+.input-group-btn:first-child > .btn-group:not(:first-child) > .btn {
+  border-bottom-left-radius: 0;
+  border-top-left-radius: 0;
+}
+.input-group-addon:last-child {
+  border-left: 0;
+}
+.input-group-btn {
+  position: relative;
+  font-size: 0;
+  white-space: nowrap;
+}
+.input-group-btn > .btn {
+  position: relative;
+}
+.input-group-btn > .btn + .btn {
+  margin-left: -1px;
+}
+.input-group-btn > .btn:hover,
+.input-group-btn > .btn:focus,
+.input-group-btn > .btn:active {
+  z-index: 2;
+}
+.input-group-btn:first-child > .btn,
+.input-group-btn:first-child > .btn-group {
+  margin-right: -1px;
+}
+.input-group-btn:last-child > .btn,
+.input-group-btn:last-child > .btn-group {
+  margin-left: -1px;
+}
+.nav {
+  margin-bottom: 0;
+  padding-left: 0;
+  list-style: none;
+}
+.nav > li {
+  position: relative;
+  display: block;
+}
+.nav > li > a {
+  position: relative;
+  display: block;
+  padding: 10px 15px;
+}
+.nav > li > a:hover,
+.nav > li > a:focus {
+  text-decoration: none;
+  background-color: #ffffff;
+}
+.nav > li.disabled > a {
+  color: #999999;
+}
+.nav > li.disabled > a:hover,
+.nav > li.disabled > a:focus {
+  color: #999999;
+  text-decoration: none;
+  background-color: transparent;
+  cursor: not-allowed;
+}
+.nav .open > a,
+.nav .open > a:hover,
+.nav .open > a:focus {
+  background-color: #ffffff;
+  border-color: #158cba;
+}
+.nav .nav-divider {
+  height: 1px;
+  margin: 11.5px 0;
+  overflow: hidden;
+  background-color: #e5e5e5;
+}
+.nav > li > a > img {
+  max-width: none;
+}
+.nav-tabs {
+  border-bottom: 1px solid #e7e7e7;
+}
+.nav-tabs > li {
+  float: left;
+  margin-bottom: -1px;
+}
+.nav-tabs > li > a {
+  margin-right: 2px;
+  line-height: 1.42857143;
+  border: 1px solid transparent;
+  border-radius: 4px 4px 0 0;
+}
+.nav-tabs > li > a:hover {
+  border-color: #eeeeee #eeeeee #e7e7e7;
+}
+.nav-tabs > li.active > a,
+.nav-tabs > li.active > a:hover,
+.nav-tabs > li.active > a:focus {
+  color: #555555;
+  background-color: #ffffff;
+  border: 1px solid #e7e7e7;
+  border-bottom-color: transparent;
+  cursor: default;
+}
+.nav-tabs.nav-justified {
+  width: 100%;
+  border-bottom: 0;
+}
+.nav-tabs.nav-justified > li {
+  float: none;
+}
+.nav-tabs.nav-justified > li > a {
+  text-align: center;
+  margin-bottom: 5px;
+}
+.nav-tabs.nav-justified > .dropdown .dropdown-menu {
+  top: auto;
+  left: auto;
+}
+@media (min-width: 768px) {
+  .nav-tabs.nav-justified > li {
+    display: table-cell;
+    width: 1%;
+  }
+  .nav-tabs.nav-justified > li > a {
+    margin-bottom: 0;
+  }
+}
+.nav-tabs.nav-justified > li > a {
+  margin-right: 0;
+  border-radius: 4px;
+}
+.nav-tabs.nav-justified > .active > a,
+.nav-tabs.nav-justified > .active > a:hover,
+.nav-tabs.nav-justified > .active > a:focus {
+  border: 1px solid #e7e7e7;
+}
+@media (min-width: 768px) {
+  .nav-tabs.nav-justified > li > a {
+    border-bottom: 1px solid #e7e7e7;
+    border-radius: 4px 4px 0 0;
+  }
+  .nav-tabs.nav-justified > .active > a,
+  .nav-tabs.nav-justified > .active > a:hover,
+  .nav-tabs.nav-justified > .active > a:focus {
+    border-bottom-color: #ffffff;
+  }
+}
+.nav-pills > li {
+  float: left;
+}
+.nav-pills > li > a {
+  border-radius: 4px;
+}
+.nav-pills > li + li {
+  margin-left: 2px;
+}
+.nav-pills > li.active > a,
+.nav-pills > li.active > a:hover,
+.nav-pills > li.active > a:focus {
+  color: #ffffff;
+  background-color: #158cba;
+}
+.nav-stacked > li {
+  float: none;
+}
+.nav-stacked > li + li {
+  margin-top: 2px;
+  margin-left: 0;
+}
+.nav-justified {
+  width: 100%;
+}
+.nav-justified > li {
+  float: none;
+}
+.nav-justified > li > a {
+  text-align: center;
+  margin-bottom: 5px;
+}
+.nav-justified > .dropdown .dropdown-menu {
+  top: auto;
+  left: auto;
+}
+@media (min-width: 768px) {
+  .nav-justified > li {
+    display: table-cell;
+    width: 1%;
+  }
+  .nav-justified > li > a {
+    margin-bottom: 0;
+  }
+}
+.nav-tabs-justified {
+  border-bottom: 0;
+}
+.nav-tabs-justified > li > a {
+  margin-right: 0;
+  border-radius: 4px;
+}
+.nav-tabs-justified > .active > a,
+.nav-tabs-justified > .active > a:hover,
+.nav-tabs-justified > .active > a:focus {
+  border: 1px solid #e7e7e7;
+}
+@media (min-width: 768px) {
+  .nav-tabs-justified > li > a {
+    border-bottom: 1px solid #e7e7e7;
+    border-radius: 4px 4px 0 0;
+  }
+  .nav-tabs-justified > .active > a,
+  .nav-tabs-justified > .active > a:hover,
+  .nav-tabs-justified > .active > a:focus {
+    border-bottom-color: #ffffff;
+  }
+}
+.tab-content > .tab-pane {
+  display: none;
+}
+.tab-content > .active {
+  display: block;
+}
+.nav-tabs .dropdown-menu {
+  margin-top: -1px;
+  border-top-right-radius: 0;
+  border-top-left-radius: 0;
+}
+.navbar {
+  position: relative;
+  min-height: 50px;
+  margin-bottom: 25px;
+  border: 1px solid transparent;
+}
+@media (min-width: 768px) {
+  .navbar {
+    border-radius: 4px;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-header {
+    float: left;
+  }
+}
+.navbar-collapse {
+  max-height: 340px;
+  overflow-x: visible;
+  padding-right: 15px;
+  padding-left: 15px;
+  border-top: 1px solid transparent;
+  box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1);
+  -webkit-overflow-scrolling: touch;
+}
+.navbar-collapse.in {
+  overflow-y: auto;
+}
+@media (min-width: 768px) {
+  .navbar-collapse {
+    width: auto;
+    border-top: 0;
+    box-shadow: none;
+  }
+  .navbar-collapse.collapse {
+    display: block !important;
+    height: auto !important;
+    padding-bottom: 0;
+    overflow: visible !important;
+  }
+  .navbar-collapse.in {
+    overflow-y: visible;
+  }
+  .navbar-fixed-top .navbar-collapse,
+  .navbar-static-top .navbar-collapse,
+  .navbar-fixed-bottom .navbar-collapse {
+    padding-left: 0;
+    padding-right: 0;
+  }
+}
+.container > .navbar-header,
+.container-fluid > .navbar-header,
+.container > .navbar-collapse,
+.container-fluid > .navbar-collapse {
+  margin-right: -15px;
+  margin-left: -15px;
+}
+@media (min-width: 768px) {
+  .container > .navbar-header,
+  .container-fluid > .navbar-header,
+  .container > .navbar-collapse,
+  .container-fluid > .navbar-collapse {
+    margin-right: 0;
+    margin-left: 0;
+  }
+}
+.navbar-static-top {
+  z-index: 1000;
+  border-width: 0 0 1px;
+}
+@media (min-width: 768px) {
+  .navbar-static-top {
+    border-radius: 0;
+  }
+}
+.navbar-fixed-top,
+.navbar-fixed-bottom {
+  position: fixed;
+  right: 0;
+  left: 0;
+  z-index: 1030;
+}
+@media (min-width: 768px) {
+  .navbar-fixed-top,
+  .navbar-fixed-bottom {
+    border-radius: 0;
+  }
+}
+.navbar-fixed-top {
+  top: 0;
+  border-width: 0 0 1px;
+}
+.navbar-fixed-bottom {
+  bottom: 0;
+  margin-bottom: 0;
+  border-width: 1px 0 0;
+}
+.navbar-brand {
+  float: left;
+  padding: 12.5px 15px;
+  font-size: 23px;
+  line-height: 25px;
+  height: 50px;
+}
+.navbar-brand:hover,
+.navbar-brand:focus {
+  text-decoration: none;
+}
+@media (min-width: 768px) {
+  .navbar > .container .navbar-brand,
+  .navbar > .container-fluid .navbar-brand {
+    margin-left: -15px;
+  }
+}
+.navbar-toggle {
+  position: relative;
+  float: right;
+  margin-right: 15px;
+  padding: 9px 10px;
+  margin-top: 8px;
+  margin-bottom: 8px;
+  background-color: transparent;
+  background-image: none;
+  border: 1px solid transparent;
+  border-radius: 4px;
+}
+.navbar-toggle:focus {
+  outline: none;
+}
+.navbar-toggle .icon-bar {
+  display: block;
+  width: 22px;
+  height: 2px;
+  border-radius: 1px;
+}
+.navbar-toggle .icon-bar + .icon-bar {
+  margin-top: 4px;
+}
+@media (min-width: 768px) {
+  .navbar-toggle {
+    display: none;
+  }
+}
+.navbar-nav {
+  margin: 6.25px -15px;
+}
+.navbar-nav > li > a {
+  padding-top: 10px;
+  padding-bottom: 10px;
+  line-height: 25px;
+}
+@media (max-width: 767px) {
+  .navbar-nav .open .dropdown-menu {
+    position: static;
+    float: none;
+    width: auto;
+    margin-top: 0;
+    background-color: transparent;
+    border: 0;
+    box-shadow: none;
+  }
+  .navbar-nav .open .dropdown-menu > li > a,
+  .navbar-nav .open .dropdown-menu .dropdown-header {
+    padding: 5px 15px 5px 25px;
+  }
+  .navbar-nav .open .dropdown-menu > li > a {
+    line-height: 25px;
+  }
+  .navbar-nav .open .dropdown-menu > li > a:hover,
+  .navbar-nav .open .dropdown-menu > li > a:focus {
+    background-image: none;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-nav {
+    float: left;
+    margin: 0;
+  }
+  .navbar-nav > li {
+    float: left;
+  }
+  .navbar-nav > li > a {
+    padding-top: 12.5px;
+    padding-bottom: 12.5px;
+  }
+  .navbar-nav.navbar-right:last-child {
+    margin-right: -15px;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-left {
+    float: left !important;
+  }
+  .navbar-right {
+    float: right !important;
+  }
+}
+.navbar-form {
+  margin-left: -15px;
+  margin-right: -15px;
+  padding: 10px 15px;
+  border-top: 1px solid transparent;
+  border-bottom: 1px solid transparent;
+  -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);
+  box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);
+  margin-top: 4.5px;
+  margin-bottom: 4.5px;
+}
+@media (min-width: 768px) {
+  .navbar-form .form-group {
+    display: inline-block;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .navbar-form .form-control {
+    display: inline-block;
+    width: auto;
+    vertical-align: middle;
+  }
+  .navbar-form .input-group > .form-control {
+    width: 100%;
+  }
+  .navbar-form .control-label {
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .navbar-form .radio,
+  .navbar-form .checkbox {
+    display: inline-block;
+    margin-top: 0;
+    margin-bottom: 0;
+    padding-left: 0;
+    vertical-align: middle;
+  }
+  .navbar-form .radio input[type="radio"],
+  .navbar-form .checkbox input[type="checkbox"] {
+    float: none;
+    margin-left: 0;
+  }
+  .navbar-form .has-feedback .form-control-feedback {
+    top: 0;
+  }
+}
+@media (max-width: 767px) {
+  .navbar-form .form-group {
+    margin-bottom: 5px;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-form {
+    width: auto;
+    border: 0;
+    margin-left: 0;
+    margin-right: 0;
+    padding-top: 0;
+    padding-bottom: 0;
+    -webkit-box-shadow: none;
+    box-shadow: none;
+  }
+  .navbar-form.navbar-right:last-child {
+    margin-right: -15px;
+  }
+}
+.navbar-nav > li > .dropdown-menu {
+  margin-top: 0;
+  border-top-right-radius: 0;
+  border-top-left-radius: 0;
+}
+.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu {
+  border-bottom-right-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.navbar-btn {
+  margin-top: 4.5px;
+  margin-bottom: 4.5px;
+}
+.navbar-btn.btn-sm {
+  margin-top: 7px;
+  margin-bottom: 7px;
+}
+.navbar-btn.btn-xs {
+  margin-top: 14px;
+  margin-bottom: 14px;
+}
+.navbar-text {
+  margin-top: 12.5px;
+  margin-bottom: 12.5px;
+}
+@media (min-width: 768px) {
+  .navbar-text {
+    float: left;
+    margin-left: 15px;
+    margin-right: 15px;
+  }
+  .navbar-text.navbar-right:last-child {
+    margin-right: 0;
+  }
+}
+.navbar-default {
+  background-color: #f8f8f8;
+  border-color: #e7e7e7;
+}
+.navbar-default .navbar-brand {
+  color: #222222;
+}
+.navbar-default .navbar-brand:hover,
+.navbar-default .navbar-brand:focus {
+  color: #222222;
+  background-color: transparent;
+}
+.navbar-default .navbar-text {
+  color: #555555;
+}
+.navbar-default .navbar-nav > li > a {
+  color: #333333;
+}
+.navbar-default .navbar-nav > li > a:hover,
+.navbar-default .navbar-nav > li > a:focus {
+  color: #222222;
+  background-color: transparent;
+}
+.navbar-default .navbar-nav > .active > a,
+.navbar-default .navbar-nav > .active > a:hover,
+.navbar-default .navbar-nav > .active > a:focus {
+  color: #222222;
+  background-color: transparent;
+}
+.navbar-default .navbar-nav > .disabled > a,
+.navbar-default .navbar-nav > .disabled > a:hover,
+.navbar-default .navbar-nav > .disabled > a:focus {
+  color: #eeeeee;
+  background-color: transparent;
+}
+.navbar-default .navbar-toggle {
+  border-color: #eeeeee;
+}
+.navbar-default .navbar-toggle:hover,
+.navbar-default .navbar-toggle:focus {
+  background-color: #ffffff;
+}
+.navbar-default .navbar-toggle .icon-bar {
+  background-color: #999999;
+}
+.navbar-default .navbar-collapse,
+.navbar-default .navbar-form {
+  border-color: #e7e7e7;
+}
+.navbar-default .navbar-nav > .open > a,
+.navbar-default .navbar-nav > .open > a:hover,
+.navbar-default .navbar-nav > .open > a:focus {
+  background-color: transparent;
+  color: #222222;
+}
+@media (max-width: 767px) {
+  .navbar-default .navbar-nav .open .dropdown-menu > li > a {
+    color: #333333;
+  }
+  .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover,
+  .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus {
+    color: #222222;
+    background-color: transparent;
+  }
+  .navbar-default .navbar-nav .open .dropdown-menu > .active > a,
+  .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover,
+  .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus {
+    color: #222222;
+    background-color: transparent;
+  }
+  .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a,
+  .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover,
+  .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus {
+    color: #eeeeee;
+    background-color: transparent;
+  }
+}
+.navbar-default .navbar-link {
+  color: #333333;
+}
+.navbar-default .navbar-link:hover {
+  color: #222222;
+}
+.navbar-inverse {
+  background-color: #ffffff;
+  border-color: #e6e6e6;
+}
+.navbar-inverse .navbar-brand {
+  color: #999999;
+}
+.navbar-inverse .navbar-brand:hover,
+.navbar-inverse .navbar-brand:focus {
+  color: #333333;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-text {
+  color: #999999;
+}
+.navbar-inverse .navbar-nav > li > a {
+  color: #999999;
+}
+.navbar-inverse .navbar-nav > li > a:hover,
+.navbar-inverse .navbar-nav > li > a:focus {
+  color: #333333;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-nav > .active > a,
+.navbar-inverse .navbar-nav > .active > a:hover,
+.navbar-inverse .navbar-nav > .active > a:focus {
+  color: #333333;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-nav > .disabled > a,
+.navbar-inverse .navbar-nav > .disabled > a:hover,
+.navbar-inverse .navbar-nav > .disabled > a:focus {
+  color: #eeeeee;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-toggle {
+  border-color: #eeeeee;
+}
+.navbar-inverse .navbar-toggle:hover,
+.navbar-inverse .navbar-toggle:focus {
+  background-color: #eeeeee;
+}
+.navbar-inverse .navbar-toggle .icon-bar {
+  background-color: #999999;
+}
+.navbar-inverse .navbar-collapse,
+.navbar-inverse .navbar-form {
+  border-color: #ededed;
+}
+.navbar-inverse .navbar-nav > .open > a,
+.navbar-inverse .navbar-nav > .open > a:hover,
+.navbar-inverse .navbar-nav > .open > a:focus {
+  background-color: transparent;
+  color: #333333;
+}
+@media (max-width: 767px) {
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header {
+    border-color: #e6e6e6;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu .divider {
+    background-color: #e6e6e6;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > li > a {
+    color: #999999;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus {
+    color: #333333;
+    background-color: transparent;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus {
+    color: #333333;
+    background-color: transparent;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus {
+    color: #eeeeee;
+    background-color: transparent;
+  }
+}
+.navbar-inverse .navbar-link {
+  color: #999999;
+}
+.navbar-inverse .navbar-link:hover {
+  color: #333333;
+}
+.breadcrumb {
+  padding: 8px 15px;
+  margin-bottom: 25px;
+  list-style: none;
+  background-color: #fafafa;
+  border-radius: 4px;
+}
+.breadcrumb > li {
+  display: inline-block;
+}
+.breadcrumb > li + li:before {
+  content: ">\00a0";
+  padding: 0 5px;
+  color: #999999;
+}
+.breadcrumb > .active {
+  color: #999999;
+}
+.pagination {
+  display: inline-block;
+  padding-left: 0;
+  margin: 25px 0;
+  border-radius: 4px;
+}
+.pagination > li {
+  display: inline;
+}
+.pagination > li > a,
+.pagination > li > span {
+  position: relative;
+  float: left;
+  padding: 6px 12px;
+  line-height: 1.42857143;
+  text-decoration: none;
+  color: #555555;
+  background-color: #eeeeee;
+  border: 1px solid #e2e2e2;
+  margin-left: -1px;
+}
+.pagination > li:first-child > a,
+.pagination > li:first-child > span {
+  margin-left: 0;
+  border-bottom-left-radius: 4px;
+  border-top-left-radius: 4px;
+}
+.pagination > li:last-child > a,
+.pagination > li:last-child > span {
+  border-bottom-right-radius: 4px;
+  border-top-right-radius: 4px;
+}
+.pagination > li > a:hover,
+.pagination > li > span:hover,
+.pagination > li > a:focus,
+.pagination > li > span:focus {
+  color: #555555;
+  background-color: #eeeeee;
+  border-color: #e2e2e2;
+}
+.pagination > .active > a,
+.pagination > .active > span,
+.pagination > .active > a:hover,
+.pagination > .active > span:hover,
+.pagination > .active > a:focus,
+.pagination > .active > span:focus {
+  z-index: 2;
+  color: #ffffff;
+  background-color: #158cba;
+  border-color: #127ba3;
+  cursor: default;
+}
+.pagination > .disabled > span,
+.pagination > .disabled > span:hover,
+.pagination > .disabled > span:focus,
+.pagination > .disabled > a,
+.pagination > .disabled > a:hover,
+.pagination > .disabled > a:focus {
+  color: #999999;
+  background-color: #eeeeee;
+  border-color: #e2e2e2;
+  cursor: not-allowed;
+}
+.pagination-lg > li > a,
+.pagination-lg > li > span {
+  padding: 10px 16px;
+  font-size: 23px;
+}
+.pagination-lg > li:first-child > a,
+.pagination-lg > li:first-child > span {
+  border-bottom-left-radius: 5px;
+  border-top-left-radius: 5px;
+}
+.pagination-lg > li:last-child > a,
+.pagination-lg > li:last-child > span {
+  border-bottom-right-radius: 5px;
+  border-top-right-radius: 5px;
+}
+.pagination-sm > li > a,
+.pagination-sm > li > span {
+  padding: 5px 10px;
+  font-size: 16px;
+}
+.pagination-sm > li:first-child > a,
+.pagination-sm > li:first-child > span {
+  border-bottom-left-radius: 2px;
+  border-top-left-radius: 2px;
+}
+.pagination-sm > li:last-child > a,
+.pagination-sm > li:last-child > span {
+  border-bottom-right-radius: 2px;
+  border-top-right-radius: 2px;
+}
+.pager {
+  padding-left: 0;
+  margin: 25px 0;
+  list-style: none;
+  text-align: center;
+}
+.pager li {
+  display: inline;
+}
+.pager li > a,
+.pager li > span {
+  display: inline-block;
+  padding: 5px 14px;
+  background-color: #eeeeee;
+  border: 1px solid #e2e2e2;
+  border-radius: 15px;
+}
+.pager li > a:hover,
+.pager li > a:focus {
+  text-decoration: none;
+  background-color: #eeeeee;
+}
+.pager .next > a,
+.pager .next > span {
+  float: right;
+}
+.pager .previous > a,
+.pager .previ

<TRUNCATED>

[51/60] Rewrite Java API Guide as Unified Programming Guide

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/97d630d5/docs/scala_api_guide.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_guide.md b/docs/scala_api_guide.md
deleted file mode 100644
index e46a898..0000000
--- a/docs/scala_api_guide.md
+++ /dev/null
@@ -1,1042 +0,0 @@
----
-title: "Scala API Programming Guide"
----
-
-<section id="top">
-Scala Programming Guide
-=======================
-
-This guide explains how to develop Flink programs with the Scala
-programming interface. 
-
-Here we will look at the general structure of a Scala job. You will learn how to
-write data sources, data sinks, and operators to create data flows that can be
-executed using the Flink system.
-
-Writing Scala jobs requires an understanding of Scala, there is excellent
-documentation available [here](http://scala-lang.org/documentation/). Most
-of the examples can be understood by someone with a good understanding
-of programming in general, though.
-
-[Back to top](#top)
-
-<section id="intro-example">
-Word Count Example
-------------------
-
-To start, let's look at a Word Count job implemented in Scala. This program is
-very simple but it will give you a basic idea of what a Scala job looks like.
-
-```scala
-import org.apache.flinkclient.LocalExecutor
-
-import org.apache.flinkapi.scala._
-import org.apache.flinkapi.scala.operators._
-
-object WordCount {
-  def main(args: Array[String]) {
-    val input = TextFile(textInput)
-
-    val words = input.flatMap { _.split(" ") map { (_, 1) } }
-
-    val counts = words.groupBy { case (word, _) => word }
-      .reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
-
-    val output = counts.write(wordsOutput, CsvOutputFormat())
-    val plan = new ScalaPlan(Seq(output))
-
-    LocalExecutor.execute(plan)
-  }
-}
-``` 
-
-Same as any Flink job a Scala job consists of one or several data
-sources, one or several data sinks and operators in between these that transform
-data. Together these parts are referred to as the data flow graph. It dictates
-the way data is passed when a job is executed.
-
-When using Scala in Flink an important concept to grasp is that of the
-`DataSet`. `DataSet` is an abstract concept that represents actual data sets at
-runtime and which has operations that transform data to create a new transformed
-data set. In this example the `TextFile("/some/input")` call creates a
-`DataSet[String]` that represents the lines of text from the input. The
-`flatMap` operation that looks like a regular Scala flatMap is in fact an
-operation on `DataSet` that passes (at runtime) the data items through the
-provided anonymous function to transform them. The result of the `flatMap`
-operation is a new `DataSet` that represents the transformed data. On this other
-operations be performed. Another such operation are `groupBy` and `reduce`, but
-we will go into details of those later in this guide.
-
-The `write` operation of `DataSet` is used to create a data sink. You provide it
-with a path where the data is to be written to and an output format. This is
-enough for now but we will discuss data formats (for sources and sinks) later.
-
-To execute a data flow graph one or several sinks have to wrapped in a `Plan`
-which can then be executed on a cluster using `RemoteExecutor`. Here, the
-`LocalExecutor` is used to run the flow on the local computer. This is useful
-for debugging your job before running it on an actual cluster.
-
-[Back to top](#top)
-
-<section id="intro-example">
-Project Setup
--------------
-
-We will only cover maven here but the concepts should work equivalently with
-other build systems such as Gradle or sbt. When wanting to develop a Scala job
-all that is needed as dependency is is `flink-scala` (and `flink-clients`, if
-you want to execute your jobs). So all that needs to be done is to add the
-following lines to your POM.
-
-
-```xml
-<dependencies>
-  <dependency>
-    <groupId>org.apache.flink</groupId>
-    <artifactId>flink-scala</artifactId>
-    <version>{{site.FLINK_VERSION_STABLE}}</version>
-  </dependency>
-  <dependency>
-    <groupId>org.apache.flink</groupId>
-    <artifactId>flink-clients</artifactId>
-    <version>{{site.FLINK_VERSION_STABLE}}</version>
-  </dependency>
-</dependencies>
-```
-
-To quickly get started you can use the Flink Scala quickstart available
-[here](scala_api_quickstart.html). This will give you a
-completeMaven project with some working example code that you can use to explore
-the system or as basis for your own projects.
-
-These imports are normally enough for any project:
-
-```scala
-import org.apache.flinkapi.scala._
-import org.apache.flinkapi.scala.operators._
-
-import org.apache.flinkclient.LocalExecutor
-import org.apache.flinkclient.RemoteExecutor
-```
-
-The first two imports contain things like `DataSet`, `Plan`, data sources, data
-sinks, and the operations. The last two imports are required if you want to run
-a data flow on your local machine, respectively cluster.
-
-[Back to top](#top)
-
-<section id="dataset">
-The DataSet Abstraction
------------------------
-
-As already alluded to in the introductory example you write Scala jobs by using
-operations on a `DataSet` to create new transformed `DataSet`. This concept is
-the core of the Flink Scala API so it merits some more explanation. A
-`DataSet` can look and behave like a regular Scala collection in your code but
-it does not contain any actual data but only represents data. For example: when
-you use `TextFile()` you get back a `DataSource[String]` that represents each
-line of text in the input as a `String`. No data is actually loaded or available
-at this point. The set is only used to apply further operations which themselves
-are not executed until the data flow is executed. An operation on `DataSet`
-creates a new `DataSet` that represents the transformation and has a pointer to
-the `DataSet` that represents the data to be transformed. In this way a tree of
-data sets is created that contains both the specification of the flow of data as
-well as all the transformations. This graph can be wrapped in a `Plan` and
-executed.
-
-Working with the system is like working with lazy collections, where execution
-is postponed until the user submits the job.
-
-`DataSet` has a generic parameter, this is the type of each data item or record
-that would be processed by further transformations. This is similar to how
-`List[A]` in Scala would behave. For example in:
-
-```scala
-val input: DataSet[(String, Int)] = ...
-val mapped = input map { a => (a._1, a._2 + 1)}
-```
-
-The anonymous function would retrieve in `a` tuples of type `(String, Int)`.
-
-[Back to top](#top)
-
-<section id="datatypes">
-Data Types
-----------
-
-There are some restrictions regarding the data types that can be used in Scala
-jobs (basically the generic parameter of `DataSet`). The usable types are
-the primitive Scala types, case classes (which includes tuples), and custom
-data types.
-
-Custom data types must implement the interface
-{% gh_link /flink-core/src/main/java/org/apache/flink/types/Value.java "Value" %}.
-For custom data types that should also be used as a grouping key or join key
-the {% gh_link /flink-core/src/main/java/org/apache/flink/types/Key.java "Key" %}
-interface must be implemented.
-
-[Back to top](#top)
-
-<section id="data-sources">
-Creating Data Sources
----------------------
-
-To get an initial `DataSet` on which to perform operations to build a data flow
-graph the following construct is used:
-
-```scala
-val input = DataSource("<file-path>", <input-format>)
-```
-
-The value `input` is now a `DataSet` with the generic type depending on the
-input format.
-
-The file path can be on of either `file:///some/file` to acces files on the
-local machine or `hdfs://some/path` to read files from HDFS. The input
-format can be one of our builtin formats or a custom input format. The builtin
-formats are:
-
-* [TextInputFormat](#text-input-format)
-* [CsvInputFormat](#csv-input-format)
-* [DelimitedInputFormat](#delimited-input-format)
-* [BinaryInputFormat](#binary-input-format)
-* [BinarySerializedInputFormat](#binary-serialized-input-format)
-* [FixedLengthInputFormat](#fixed-length-input-format)
-
-We will now have a look at each of them and show how they are employed and in
-which situations.
-
-[Back to top](#top)
-
-<section id="text-input-format">
-#### TextInputFormat
-
-This input format simply reads a text file line wise and creates a `String`
-for each line. It is used as:
-
-```scala
-TextInputFormat()
-```
-
-As you have already seen in the Word Count Example there is a shortcut for this.
-Instead of using a `DataSource` with `TextInputFormat` you can simply write:
-
-```scala
-val input = TextFile("<file-path>")
-```
-
-The `input` would then be a `DataSet[String]`.
-
-[Back to top](#top)
-
-<section id="csv-input-format">
-#### CsvInputFormat
-
-This input format is mainly used to read Csv-Files, as the name suggests. Input
-files must be text files. You can specify the `String` that should be used
-as the separator between individual records (this would often be newline) and
-also the separator between fields of a record (this would often be a comma).
-The `CsvInputFormat` will automatically read the records and create
-Scala tuples or custom case class objects for you. The format can be used
-in one of the following ways:
-
-```scala
-CsvInputFormat[Out]()
-CsvInputFormat[Out](recordDelim: String)
-CsvInputFormat[Out](recordDelim: String, fieldDelim: Char)
-
-CsvInputFormat[Out](fieldIndices: Seq[Int])
-CsvInputFormat[Out](fieldIndices: Seq[Int], recordDelim: String)
-CsvInputFormat[Out](fieldIndices: Seq[Int], recordDelim: String, fieldDelim: Char)
-```
-
-The default record delimiter is a newline, the default field delimiter is a
-comma. The type parameter `Out` must be a case class type, which also includes
-tuple types since they are internally case classes.
-
-Normally, all the fields of a record are read. If you want to explicitly
-specify which fields of the record should be read you can use one of the
-tree variants with a `fieldIndices` parameter. Here you give a list
-of the fields that should be read. Field indices start from zero.
-
-An example usage could look as follows:
-
-```scala
-val input = DataSource("file:///some/file", CsvInputFormat[(Int, Int, String)](Seq(1, 17, 42), "\n", ','))
-```
-
-Here only the specified fields would be read and 3-tuples created for you.
-The type of input would be `DataSet[(Int, Int, String)]`.
-
-[Back to top](#top)
-
-<section id="delimited-input-format">
-#### DelimitedInputFormat
-
-This input format is meant for textual records that are separated by
-some delimiter. The delimiter could be a newline, for example. It is used like
-this:
-
-```scala
-DelimitedInputFormat[Out](parseFunction: String => Out, delim: String = "\n")
-```
-
-The input files will be split on the supplied delimiter (or the default newline)
-and the supplied parse function must parse the textual representation in the
-`String` and return an object. The type of this object will then also be the
-type of the `DataSet` created by the `DataSource` operation.
-
-Just as with `BinaryInputFormat` the function can be an anonymous function, so
-you could have:
-
-```scala
-val input = DataSource("file:///some/file", BinaryInputFormat( { line =>
-  line match {
-    case EdgeInputPattern(from, to) => Path(from.toInt, to.toInt, 1)
-  }
-}))
-```
-
-In this example EdgeInputPattern is some regular expression used for parsing
-a line of text and `Path` is a custom case class that is used to represent
-the data. The type of input would in this case be `DataSet[Path]`.
-
-[Back to top](#top)
-
-<section id="binary-input-format">
-#### BinaryInputFormat
-
-This input format is best used when you have a custom binary format that
-you store the data in. It is created using one of the following:
-
-```scala
-BinaryInputFormat[Out](readFunction: DataInput => Out)
-BinaryInputFormat[Out](readFunction: DataInput => Out, blocksize: Long)
-```
-
-So you have to provide a function that gets a
-[java.io.DataInput](http://docs.oracle.com/javase/7/docs/api/java/io/DataInput.html)
-and returns the object that
-contains the data. The type of this object will then also be the type of the
-`DataSet` created by the `DataSource` operation.
-
-The provided function can also be an anonymous function, so you could
-have something like this:
-
-```scala
-val input = DataSource("file:///some/file", BinaryInputFormat( { input =>
-  val one = input.readInt
-  val two = input.readDouble
-  (one, two)  
-}))
-```
-
-Here `input` would be of type `DataSet[(Int, Double)]`.
-
-[Back to top](#top)
-
-<section id="binary-serialized-input-format">
-#### BinarySerializedInputFormat
-
-This input format is only meant to be used in conjunction with
-`BinarySerializedOutputFormat`. You can use these to write elements to files using a
-Flink-internal format that can efficiently be read again. You should only
-use this when output is only meant to be consumed by other Flink jobs.
-The format can be used on one of two ways:
-
-```scala
-BinarySerializedInputFormat[Out]()
-BinarySerializedInputFormat[Out](blocksize: Long)
-```
-
-So if input files contain elements of type `(String, Int)` (a tuple type) you
-could use:
-
-```scala
-val input = DataSource("file:///some/file", BinarySerializedInputFormat[(String, Int)]())
-```
-[Back to top](#top)
-
-<section id="fixed-length-input-format">
-#### FixedLengthInputFormat
-
-This input format is for cases where you want to read binary blocks
-of a fixed size. The size of a block must be specified and you must
-provide code that reads elements from a byte array.
-
-The format is used like this:
-
-```scala
-FixedLengthInputFormat[Out](readFunction: (Array[Byte], Int) => Out, recordLength: Int)
-```
-
-The specified function gets an array and a position at which it must start
-reading the array and returns the element read from the binary data.
-
-[Back to top](#top)
-
-<section id="operations">
-Operations on DataSet
----------------------
-
-As explained in [Java API](java_api_guide.html#transformations),
-a Flink job is a graph of operators that process data coming from
-sources that is finally written to sinks. When you use the Scala front end
-these operators as well as the graph is created behind the scenes. For example,
-when you write code like this:
-
-```scala
-val input = TextFile("file:///some/file")
-
-val words = input.map { x => (x, 1) }
-
-val output = counts.write(words, CsvOutputFormat()))
-
-val plan = new ScalaPlan(Seq(output))
-```
-
-What you get is a graph that has a data source, a map operator (that contains
-the code written inside the anonymous function block), and a data sink. You 
-do not have to know about this to be able to use the Scala front end but
-it helps to remember, that when you are using Scala you are building
-a data flow graph that processes data only when executed.
-
-There are operations on `DataSet` that correspond to all the types of operators
-that the Flink system supports. We will shortly go trough all of them with
-some examples.
-
-[Back to top](#top)
-
-<section id="operator-templates">
-#### Basic Operator Templates
-
-Most of the operations have three similar versions and we will
-explain them here for all of the operators together. The three versions are `map`,
-`flatMap`, and `filter`. All of them accept an anonymous function that
-defines what the operation does but the semantics are different.
-
-The `map` version is a simple one to one mapping. Take a look at the following
-code:
-
-```scala
-val input: DataSet[(String, Int)]
-
-val mapped = input.map { x => (x._1, x._2 + 3) }
-```
-
-This defines a map operator that operates on tuples of String and Int and just
-adds three to the Int (the second fields of the tuple). So, if the input set had
-the tuples (a, 1), (b, 2), and (c, 3) the result after the operator would be
-(a, 4), (b, 5), and (c, 6).
-
-The `flatMap` version works a bit differently,
-here you return something iterable from the anonymous function. The iterable
-could be a list or an array. The elements in this iterable are unnested.
-So for every element in the input data you get a list of elements. The
-concatenation of those is the result of the operator. If you had
-the following code:
-
-```scala
-val input: DataSet[(String, Int)]
-
-val mapped = input.flatMap { x => List( (x._1, x._2), (x._1, x._2 + 1) ) }
-```
-
-and as input the tuples (a, 1) and (b, 1) you would get (a, 1), (a, 2), (b, 1),
-and (b, 2) as result. It is one flat list, and not the individual lists returned
-from the anonymous function.
-
-The third template is `filter`. Here you give an anonymous function that
-returns a Boolean. The elements for which this Boolean is true are part of the
-result of the operation, the others are culled. An example for a filter is this
-code:
-
-
-```scala
-val input: DataSet[(String, Int)]
-
-val mapped = input.filter { x => x._2 >= 3 }
-```
-[Back to top](#top)
-
-<section id="key-selectors">
-#### Field/Key Selectors
-
-For some operations (group, join, and cogroup) it is necessary to specify which
-parts of a data type are to be considered the key. This key is used for grouping
-elements together for reduce and for joining in case of a join or cogroup.
-In Scala the key is specified using a special anonymous function called
-a key selector. The key selector has as input an element of the type of
-the `DataSet` and must return a single value or a tuple of values that should
-be considered the key. This will become clear with some examples: (Note that
-we use the reduce operation here as an example, we will have a look at
-that further down):
-
-```scala
-val input: DataSet[(String, Int)]
-val reduced = input groupBy { x => (x._1) } reduce { ... }
-val reduced2 = input groupBy { case (w, c) => w } reduce { ... }
-
-case class Test(a: String, b: Int, c: Int)
-val input2: DataSet[Test]
-val reduced3 = input2 groupBy { x => (x.a, x.b) } reduce { ... }
-val reduced4 = input2 groupBy { case Test(x,y,z) => (x,y) } reduce { ... }
-```
-
-The anonymous function block passed to `groupBy` is the key selector. The first
-two examples both specify the `String` field of the tuple as key. In the second
-set of examples we see a custom case class and here we select the first two
-fields as a compound key.
-
-It is worth noting that the key selector function is not actually executed 
-at runtime but it is parsed at job creation time where the key information is
-extracted and stored for efficient computation at runtime.
-
-#### Map Operation
-
-Map is an operation that gets one element at a time and can output one or
-several elements. The operations that result in a `MapOperator` in the graph are exactly
-those mentioned in the previous section. For completeness' sake we will mention
-their signatures here (in this and the following such lists `In` is the
-type of the input data set, `DataSet[In]`):
-
-```scala
-def map[Out](fun: In => Out): DataSet[Out]
-def flatMap[Out](fun: In => Iterator[Out]): DataSet[Out]
-def filter(fun: In => Boolean): DataSet[Out]
-```
-
-#### Reduce Operation
-
-Reduce is an operation that looks
-at groups of elements at a time and can, for one group, output one or several
-elements. To specify how elements should be grouped you need to give
-a key selection function, as explained [above](#key-selectors).
-
-The basic template of the reduce operation is:
-
-```scala
-input groupBy { <key selector> } reduce { <reduce function> }
-```
-
-The signature of the reduce function depends on the variety of reduce operation
-selected. There are right now three different versions:
-
-```scala
-def reduce(fun: (In, In) => In): DataSet[In]
-
-def reduceGroup[Out](fun: Iterator[In] => Out): DataSet[Out]
-def combinableReduceGroup(fun: Iterator[In] => In): DataSet[In]
-```
-
-The `reduce` variant is like a `reduceLeft` on a Scala collection with
-the limitation that the output data type must be the same as the input data
-type. You specify how to elements of the selection should be combined,
-this is then used to reduce the elements in one group (of the same key)
-down to one element. This can be used to implement aggregation operators,
-for example:
-
-```scala
-val words: DataSet[(String, Int)]
-val counts = words.groupBy { case (word, count) => word}
-  .reduce { (w1, w1) => (w1._1, w1._2 + w2._2) }
-```
-
-This would add up the Int fields of those tuples that have the same String
-in the first fields. As is for example required in Word Count.
-
-The `reduceGroup` variant can be used when more control is required. Here
-your reduce function gets an `Iterator` that can be used to iterate over
-all the elements in a group. With this type or reduce operation the
-output data type can be different from the input data type. An example
-of this kind of operation is this:
-
-```scala
-val words: DataSet[(String, Int)]
-val minCounts = words.groupBy { case (word, count) => word}
-  .reduceGroup { words => words.minBy { _._2 } }
-```
-
-Here we use the minBy function of Scala collections to determine the
-element with the minimum count in a group.
-
-The `combinableGroupReduce` works like the `groupReduce` with the difference
-that the reduce operation is combinable. This is an optimization one can use,
-please have a look at [Programming Model](pmodel.html "Programming Model") for
-the details.
-
-#### Join Operation
-
-The join operation is similar to a database equi-join. It is a two input
-iteration where you have to specify a key selector for each of the inputs
-and then the anonymous function is called for every pair of matching
-elements from the two input sides.
-
-The basic template is:
-
-```scala
-input1 join input2 where { <key selector 1> } isEqualTo { <key selector 2>} map { <join function> }
-```
-
-or, because lines will get to long fast:
-```scala
-input1.join(input2)
-  .where { <key selector 1> }
-  .isEqualTo { <key selector 2>}
-  .map { <join function> }
-```
-
-(Scala can sometimes be quite finicky about where you can omit dots and
-parentheses, so it's best to use dots in multi-line code like this.)
-
-As mentioned in [here](#operator-templates) there are three versions of
-this operator, so you can use one of these in the last position:
-
-```scala
-def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out]
-def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out]
-def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)]
-```
-
-One example where this can be used is database-style joining with projection:
-
-```scala
-input1.join(input2)
-  .where { case (a, b, c) => (a, b) }
-  .isEqualTo { case (a, b, c, d) => (c, d) }
-  .map { (left, right) => (left._3, right._1) }
-```
-
-Here the join key for the left input is a compound of the first two tuple fields
-while the key for the second input is a compound of the last two fields. We then
-pick one field each from both sides as the result of the operation.
-
-#### CoGroup Operation
-
-The cogroup operation is a cross between join and reduce. It has two inputs
-and you have to specify a key selector for each of them. This is where the
-similarities with join stop. Instead of having one invocation of your user
-code per pair of matching elements all elements from the left and from the right
-are grouped together for one single invocation. In your function you get
-an `Iterator` for the elements from the left input and another `Iterator`
-for the elements from the right input.
-
-The basic template is:
-
-```scala
-input1 cogroup input2 where { <key selector 1> } isEqualTo { <key selector 2>} map { <cogroup function> }
-```
-
-or, because lines will get to long fast:
-```scala
-input1.cogroup(input2)
-  .where { <key selector 1> }
-  .isEqualTo { <key selector 2>}
-  .map { <cogroup function> }
-```
-
-There are to variants you can use, with the semantics explained
-[here](#operator-templates).
-
-```scala
-def map[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Out): DataSet[Out]
-def flatMap[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Iterator[Out]): DataSet[Out]
-```
-
-#### Cross Operation
-
-The cross operation is used to form the Cartesian product of the elements
-from two inputs. The basic template is:
-
-```scala
-input1 cross input2 map { <cogroup function> }
-```
-
-Again there are three variants, with the semantics explained
-[here](#operator-templates).
-
-```scala
-def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out]
-def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out]
-def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)]
-```
-
-#### Union
-
-When you want to have the combination of several data sets as the input of
-an operation you can use a union to combine them. It is used like this
-
-```scala
-val input1: DataSet[String]
-val input2: DataSet[String]
-val unioned = input1.union(input2)
-```
-
-The signature of union is:
-
-```scala
-def union(secondInput: DataSet[A])
-```
-
-Where `A` is the generic type of the `DataSet` on which you execute the `union`.
-
-[Back to top](#top)
-
-<section id="iterations">
-Iterations
-----------
-
-Iterations allow you to implement *loops* in Flink programs.
-[This page](iterations.html) gives a
-general introduction to iterations. This section here provides quick examples
-of how to use the concepts using the Scala API.
-The iteration operators encapsulate a part of the program and execute it
-repeatedly, feeding back the result of one iteration (the partial solution) into
-the next iteration. Flink has two different types of iterations,
-*Bulk Iteration* and *Delta Iteration*.
-
-For both types of iterations you provide the iteration body as a function
-that has data sets as input and returns a new data set. The difference is
-that bulk iterations map from one data set two one new data set while
-delta iterations map two data sets to two new data sets.
-
-#### Bulk Iteration
-
-The signature of the bulk iterate method is this:
-
-```scala
-def iterate(n: Int, stepFunction: DataSet[A] => DataSet[A])
-```
-
-where `A` is the type of the `DataSet` on which `iterate` is called. The number
-of steps is given in `n`. This is how you use it in practice:
-
-```scala
-val dataPoints = DataSource(dataPointInput, DelimitedInputFormat(parseInput))
-val clusterPoints = DataSource(clusterInput, DelimitedInputFormat(parseInput))
-
-def kMeansStep(centers: DataSet[(Int, Point)]) = {
-
-  val distances = dataPoints cross centers map computeDistance
-  val nearestCenters = distances.groupBy { case (pid, _) => pid }
-    .reduceGroup { ds => ds.minBy(_._2.distance) } map asPointSum.tupled
-  val newCenters = nearestCenters.groupBy { case (cid, _) => cid }
-    .reduceGroup sumPointSums map { case (cid, pSum) => cid -> pSum.toPoint() }
-
-  newCenters
-}
-
-val finalCenters = clusterPoints.iterate(numIterations, kMeansStep)
-
-val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatOutput.tupled))
-```
-
-Not that we use some functions here which we don't show. If you want, you
-can check out the complete code in our KMeans example.
-
-#### Delta Iteration
-
-The signature of the delta iterate method is this:
-
-```scala
-def iterateWithDelta(workset: DataSet[W], solutionSetKey: A => K, stepFunction: (DataSet[A], DataSet[W]) => (DataSet[A], DataSet[W]), maxIterations: Int)
-```
-
-where `A` is the type of the `DataSet` on which `iterateWithDelta` is called,
-`W` is the type of the `DataSet` that represents the workset and `K` is the
-key type. The maximum number of iterations must always be given.
-
-For information on how delta iterations in general work on our system, please
-refer to [iterations](iterations.html). A working example job is
-available here:
-[Scala Connected Components Example](examples_scala.html#connected_components) 
-
-[Back to top](#top)
-
-<section id="data-sinks">
-Creating Data Sinks
--------------------
-
-The creation of data sinks is analog to the creation of data sources. `DataSet`
-has a `write` method that is used to create a sink that writes the output
-of the operation to a file in the local file system or HDFS. The general pattern
-is this:
-
-```scala
-val sink = out.write("<file-path>", <output-format>)
-```
-
-Where `out` is some `DataSet`. Just as for data sources, the file path can be
-on of either `file:///some/file` to acces files on the local machine or
-`hdfs://some/path` to read files from HDFS. The output format can be one of our
-builtin formats or a custom output format. The builtin formats are:
-
-* [DelimitedOutputFormat](#delimited-output-format)
-* [CsvOutputFormat](#csv-output-format)
-* [RawOutputFormat](#raw-output-format)
-* [BinaryOutputFormat](#binary-output-format)
-* [BinarySerializedOutputFormat](#binary-serialized-output-format)
-
-We will now have a look at each of them and show how they are employed and in
-which situations.
-
-[Back to top](#top)
-
-<section id="delimited-output-format">
-#### DelimitedOutputFormat
-
-This output format is meant for writing textual records that are separated by
-some delimiter. The delimiter could be a newline, for example. It is used like
-this:
-
-```scala
-DelimitedOutputFormat[In](formatFunction: In => String, delim: String = "\n")
-```
-
-For every element in the `DataSet` the formatting function is called and
-the result of that is appended to the output file. In between the elements
-the `delim` string is inserted.
-
-An example would be:
-
-```scala
-val out: DataSet[(String, Int)]
-val sink = out.write("file:///some/file", DelimitedOutputFormat( { elem =>
-  "%s|%d".format(elem._1, elem._2)
-}))
-```
-
-Here we use Scala String formatting to write the two fields of the tuple
-separated by a pipe character. The default newline delimiter will be inserted
-between the elements in the output files.
-
-[Back to top](#top)
-
-<section id="csv-output-format">
-#### CsvOutputFormat
-
-This output format can be used to automatically write fields of tuple
-elements or case classes to CSV files. You can specify what separator should
-be used between fields of an element and also the separator between elements.
-
-```scala
-CsvOutputFormat[In]()
-CsvOutputFormat[In](recordDelim: String)
-CsvOutputFormat[In](recordDelim: String, fieldDelim: Char)
-```
-
-The default record delimiter is a newline, the default field delimiter is a
-comma. 
-
-An example usage could look as follows:
-
-```scala
-val out: DataSet[(String, Int)]
-val sink = out.write("file:///some/file", CsvOutputFormat())
-```
-
-Notice how we don't need to specify the generic type here, it is inferred.
-
-[Back to top](#top)
-
-<section id="raw-output-format">
-#### RawOutputFormat
-
-This input format can be used when you want to have complete control over
-what gets written. You get an
-[OutputStream](http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html)
-and can write the elements of the `DataSet` exactly as you see fit.
-
-A `RawOutputFormat` is created like this:
-
-```scala
-RawOutputFormat[In](writeFunction: (In, OutputStream) => Unit)
-```
-
-The function you pass in gets one element from the `DataSet` and must
-write it to the given `OutputStream`. An example would be the following:
-
-```scala
-val out: DataSet[(String, Int)]
-val sink = out.write("file:///some/file", RawOutputFormat( { (elem, output) =>
-  /* write elem._1 and elem._2 to output */ 
-}))
-```
-
-<section id="binary-output-format">
-#### BinaryOutputFormat
-
-This format is very similar to the RawOutputFormat. The difference is that
-instead of an [OutputStream](http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html)
-you get a [DataOutput](http://docs.oracle.com/javase/7/docs/api/java/io/DataOutput.html)
-to which you can write binary data. You can also specify the block size for
-the binary output file. When you don't specify a block size some default
-is used.
-
-A `BinaryOutputFormat` is created like this:
-
-```scala
-BinaryOutputFormat[In](writeFunction: (In, DataOutput) => Unit)
-BinaryOutputFormat[In](writeFunction: (In, DataOutput) => Unit, blockSize: Long)
-```
-[Back to top](#top)
-
-<section id="binary-serialized-output-format">
-#### BinarySerializedOutputFormat
-
-This output format is only meant to be used in conjunction with
-`BinarySerializedInputFormat`. You can use these to write elements to files using a
-Flink-internal format that can efficiently be read again. You should only
-use this when output is only meant to be consumed by other Flink jobs.
-The output format can be used on one of two ways:
-
-```scala
-BinarySerializedOutputFormat[In]()
-BinarySerializedOutputFormat[In](blocksize: Long)
-```
-
-So to write elements of some `DataSet[A]` to a binary file you could use:
-
-```scala
-val out: DataSet[(String, Int)]
-val sink = out.write("file:///some/file", BinarySerializedInputFormat())
-```
-
-As you can see the type of the elements need not be specified, it is inferred
-by Scala.
-
-[Back to top](#top)
-
-<section id="execution">
-Executing Jobs
---------------
-
-To execute a data flow graph the sinks need to be wrapped in a {% gh_link /flink-scala/src/main/scala/org/apache/flink/api/scala/ScalaPlan.scala "ScalaPlan" %} object like this:
-
-```scala
-val out: DataSet[(String, Int)]
-val sink = out.write("file:///some/file", CsvOutputFormat())
-
-val plan = new ScalaPlan(Seq(sink))
-```
-
-You can put several sinks into the `Seq` that is passed to the constructor.
-
-There are two ways one can execute a data flow plan: local execution and
-remote/cluster execution. When using local execution the plan is executed on
-the local computer. This is handy while developing jobs because you can
-easily debug your code and iterate quickly. When a job is ready to be
-used on bigger data sets it can be executed on a cluster. We will
-now give an example for each of the two execution modes.
-
-First up is local execution:
-
-```scala
-import org.apache.flinkclient.LocalExecutor
-
-...
-
-val plan: ScalaPlan = ...
-LocalExecutor.execute(plan)
-```
-
-This is all there is to it.
-
-Remote (or cluster) execution is a bit more complicated because you have
-to package your code in a jar file so that it can be distributed on the cluster.
-Have a look at the [scala quickstart](scala_api_quickstart.html) to see how you
-can set up a maven project that does the packaging. Remote execution is done
-using the {% gh_link /flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java "RemoteExecutor" %}, like this:
-
-```scala
-import org.apache.flinkclient.RemoteExecutor
-
-...
-
-val plan: ScalaPlan = ...
-val ex = new RemoteExecutor("<job manager ip address>", <job manager port>, "your.jar");
-ex.executePlan(plan);
-```
-
-The IP address and the port of the Flink job manager depend on your
-setup. Have a look at [cluster quickstart](/quickstart/setup.html) for a quick
-guide about how to set up a cluster. The default cluster port is 6123, so
-if you run a job manger on your local computer you can give this and "localhost"
-as the first to parameters to the `RemoteExecutor` constructor.
-
-[Back to top](#top)
-
-<section id="rich-functions">
-Rich Functions
---------------
-
-Sometimes having a single function that is passed to an operation is not enough.
-Using Rich Functions it is possible to have state inside your operations and
-have code executed before the first element is processed and after the last
-element is processed. For example, instead of a simple function as in this
-example:
-
-```scala
-val mapped = input map { x => x + 1 }
-```
-
-you can have a rich function like this:
-
-```scala
-val mapped = input map( new MapFunction[(String, Int), (String, Int)] {
-  val someState: SomeType = ...
-  override def open(config: Configuration) = {
-    // one-time initialization code
-  }
-
-  override def close() = {
-    // one-time clean-up code
-  }
-
-  override def apply(in: (String, Int)) = {
-    // do complex stuff
-    val result = ...
-    result
-  }
-})
-```
-
-You could also create a custom class that derives from `MapFunction`
-instead of the anonymous class we used here.
-
-There are rich functions for all the various operator types. The basic
-template is the some, though. The common interface that they implement 
-is {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/functions/Function.java "Function" %}. The `open` and `close` methods can be overridden to run set-up
-and tear-down code. The other methods can be used in a rich function to
-work with the runtime context which gives information about the context
-of the operator. Your operation code must now reside in an `apply` method
-that has the same signature as the anonymous function you would normally
-supply.
-
-The rich functions reside in the package `org.apache.flinkapi.scala.functions`.
-This is a list of all the rich functions can can be used instead of
-simple functions in the respective operations:
-
-```scala
-abstract class MapFunction[In, Out] 
-abstract class FlatMapFunction[In, Out] 
-abstract class FilterFunction[In, Out] 
-
-abstract class ReduceFunction[In]
-abstract class GroupReduceFunction[In, Out]
-abstract class CombinableGroupReduceFunction[In, Out]
-
-abstract class JoinFunction[LeftIn, RightIn, Out]
-abstract class FlatJoinFunction[LeftIn, RightIn, Out]
-
-abstract class CoGroupFunction[LeftIn, RightIn, Out]
-abstract class FlatCoGroupFunction[LeftIn, RightIn, Out]
-
-abstract class CrossFunction[LeftIn, RightIn, Out]
-abstract class FlatCrossFunction[LeftIn, RightIn, Out]
-```
-
-Note that for all the rich stubs, you need to specify the generic type of
-the input (or inputs) and the output type.
-
-[Back to top](#top)
\ No newline at end of file


[34/60] git commit: Adds PageRankBasic Scala example. Removes old Scala examples

Posted by al...@apache.org.
Adds PageRankBasic Scala example. Removes old Scala examples


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

Branch: refs/heads/master
Commit: 0dc761473a4bfad5f22152cec443dde7cfaee7eb
Parents: 81e81b9
Author: Fabian Hueske <fh...@apache.org>
Authored: Thu Sep 11 15:42:29 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../flink/example/java/graph/PageRankBasic.java |  23 ++-
 .../example/java/graph/util/PageRankData.java   |  83 ++++-----
 .../scala/graph/ComputeEdgeDegrees.scala        | 123 ------------
 .../flink/examples/scala/graph/PageRank.scala   | 108 -----------
 .../examples/scala/graph/PageRankBasic.scala    | 185 +++++++++++++++++++
 .../scala/graph/PageRankWithWeight.scala        | 108 -----------
 .../iterations/PageRankCompilerTest.java        |   4 +-
 7 files changed, 240 insertions(+), 394 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
index 656c424..8a57007 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
@@ -91,7 +91,7 @@ public class PageRankBasic {
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		
 		// get input data
-		DataSet<Tuple1<Long>> pagesInput = getPagesDataSet(env);
+		DataSet<Long> pagesInput = getPagesDataSet(env);
 		DataSet<Tuple2<Long, Long>> linksInput = getLinksDataSet(env);
 		
 		// assign initial rank to pages
@@ -138,7 +138,7 @@ public class PageRankBasic {
 	/** 
 	 * A map function that assigns an initial rank to all pages. 
 	 */
-	public static final class RankAssigner implements MapFunction<Tuple1<Long>, Tuple2<Long, Double>> {
+	public static final class RankAssigner implements MapFunction<Long, Tuple2<Long, Double>> {
 		Tuple2<Long, Double> outPageWithRank;
 		
 		public RankAssigner(double rank) {
@@ -146,8 +146,8 @@ public class PageRankBasic {
 		}
 		
 		@Override
-		public Tuple2<Long, Double> map(Tuple1<Long> page) {
-			outPageWithRank.f0 = page.f0;
+		public Tuple2<Long, Double> map(Long page) {
+			outPageWithRank.f0 = page;
 			return outPageWithRank;
 		}
 	}
@@ -259,12 +259,17 @@ public class PageRankBasic {
 		return true;
 	}
 	
-	private static DataSet<Tuple1<Long>> getPagesDataSet(ExecutionEnvironment env) {
+	private static DataSet<Long> getPagesDataSet(ExecutionEnvironment env) {
 		if(fileOutput) {
-			return env.readCsvFile(pagesInputPath)
-						.fieldDelimiter(' ')
-						.lineDelimiter("\n")
-						.types(Long.class);
+			return env
+						.readCsvFile(pagesInputPath)
+							.fieldDelimiter(' ')
+							.lineDelimiter("\n")
+							.types(Long.class)
+						.map(new MapFunction<Tuple1<Long>, Long>() {
+							@Override
+							public Long map(Tuple1<Long> v) { return v.f0; }
+						});
 		} else {
 			return PageRankData.getDefaultPagesDataSet(env);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
index 6e494a4..10cf748 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
@@ -22,11 +22,9 @@ package org.apache.flink.example.java.graph.util;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
 
 /**
  * Provides the default data sets used for the PageRank example program.
@@ -35,52 +33,51 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  */
 public class PageRankData {
 
+	public static final Object[][] EDGES = {
+		{1L, 2L},
+		{1L, 15L},
+		{2L, 3L},
+		{2L, 4L},
+		{2L, 5L},
+		{2L, 6L},
+		{2L, 7L},
+		{3L, 13L},
+		{4L, 2L},
+		{5L, 11L},
+		{5L, 12L},
+		{6L, 1L},
+		{6L, 7L},
+		{6L, 8L},
+		{7L, 1L},
+		{7L, 8L},
+		{8L, 1L},
+		{8L, 9L},
+		{8L, 10L},
+		{9L, 14L},
+		{9L, 1L},
+		{10L, 1L},
+		{10L, 13L},
+		{11L, 12L},
+		{11L, 1L},
+		{12L, 1L},
+		{13L, 14L},
+		{14L, 12L},
+		{15L, 1L},
+	};
+	
 	private static long numPages = 15;
 	
-	public static DataSet<Tuple1<Long>> getDefaultPagesDataSet(ExecutionEnvironment env) {
-		
-		List<Tuple1<Long>> data = new ArrayList<Tuple1<Long>>();
+	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 		
-		for(long i=0; i<numPages; i++) {
-			data.add(new Tuple1<Long>(i));
+		List<Tuple2<Long, Long>> edges = new ArrayList<Tuple2<Long, Long>>();
+		for(Object[] e : EDGES) {
+			edges.add(new Tuple2<Long, Long>((Long)e[0], (Long)e[1]));
 		}
-		return env.fromCollection(data);
+		return env.fromCollection(edges);
 	}
 	
-	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-		
-		List<Tuple2<Long, Long>> data = new ArrayList<Tuple2<Long, Long>>();
-		data.add(new Tuple2<Long, Long>(1L, 2L));
-		data.add(new Tuple2<Long, Long>(1L, 15L));
-		data.add(new Tuple2<Long, Long>(2L, 3L));
-		data.add(new Tuple2<Long, Long>(2L, 4L));
-		data.add(new Tuple2<Long, Long>(2L, 5L));
-		data.add(new Tuple2<Long, Long>(2L, 6L));
-		data.add(new Tuple2<Long, Long>(2L, 7L));
-		data.add(new Tuple2<Long, Long>(3L, 13L));
-		data.add(new Tuple2<Long, Long>(4L, 2L));
-		data.add(new Tuple2<Long, Long>(5L, 11L));
-		data.add(new Tuple2<Long, Long>(5L, 12L));
-		data.add(new Tuple2<Long, Long>(6L, 1L));
-		data.add(new Tuple2<Long, Long>(6L, 7L));
-		data.add(new Tuple2<Long, Long>(6L, 8L));
-		data.add(new Tuple2<Long, Long>(7L, 1L));
-		data.add(new Tuple2<Long, Long>(7L, 8L));
-		data.add(new Tuple2<Long, Long>(8L, 1L));
-		data.add(new Tuple2<Long, Long>(8L, 9L));
-		data.add(new Tuple2<Long, Long>(8L, 10L));
-		data.add(new Tuple2<Long, Long>(9L, 14L));
-		data.add(new Tuple2<Long, Long>(9L, 1L));
-		data.add(new Tuple2<Long, Long>(10L, 1L));
-		data.add(new Tuple2<Long, Long>(10L, 13L));
-		data.add(new Tuple2<Long, Long>(11L, 12L));
-		data.add(new Tuple2<Long, Long>(11L, 1L));
-		data.add(new Tuple2<Long, Long>(12L, 1L));
-		data.add(new Tuple2<Long, Long>(13L, 14L));
-		data.add(new Tuple2<Long, Long>(14L, 12L));
-		data.add(new Tuple2<Long, Long>(15L, 1L));
-		
-		return env.fromCollection(data);
+	public static DataSet<Long> getDefaultPagesDataSet(ExecutionEnvironment env) {
+		return env.generateSequence(1, 15);
 	}
 	
 	public static long getNumberOfPages() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
deleted file mode 100644
index 5200074..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
+++ /dev/null
@@ -1,123 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.graph
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-///**
-// * Annotates edges with associated vertex degrees.
-// */
-//class ComputeEdgeDegrees extends Program with ProgramDescription with Serializable {
-//  override def getDescription() = {
-//    "Parameters: [numSubStasks] [input file] [output file]"
-//  }
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1), args(2))
-//  }
-//
-//  /*
-//   * Output formatting function for edges with annotated degrees
-//   */
-//  def formatEdgeWithDegrees = (v1: Int, v2: Int, c1: Int, c2: Int) => "%d,%d|%d,%d".format(v1, v2, c1, c2)
-//
-//  /*
-//   * Emits one edge for each unique input edge with the vertex degree of the first(and grouping key) vertex.
-//   * The degree of the second (non-grouping key) vertexes are set to zero.
-//   * Edges are projected such that smaller vertex is the first vertex.
-//   */
-//  def annotateFirstVertexDegree(eI: Iterator[(Int, Int)]): List[(Int, Int, Int, Int)] = {
-//    val eL = eI.toList
-//    val eLUniq = eL.distinct
-//    val cnt = eLUniq.size
-//    for (e <- eLUniq)
-//      yield if (e._1 < e._2)
-//    	  		(e._1, e._2, cnt, 0)
-//        	else
-//        		(e._2, e._1, 0, cnt)
-//  }
-//
-//  /*
-//   * Combines the degrees of both vertexes of an edge.
-//   */
-//  def combineVertexDegrees(eI: Iterator[(Int, Int, Int, Int)]) : (Int, Int, Int, Int) = {
-//
-//    val eL = eI.toList
-//    if (eL.size != 2)
-//    	throw new RuntimeException("Problem when combinig vertex counts");
-//
-//    if (eL(0)._3 == 0 && eL(1)._4 == 0)
-//      (eL(0)._1, eL(1)._3, eL(0)._2, eL(0)._4)
-//    else
-//      (eL(0)._1, eL(0)._3, eL(0)._2, eL(1)._4)
-//
-//  }
-//
-//  def getScalaPlan(numSubTasks: Int, edgeInput: String, annotatedEdgeOutput: String) = {
-//
-//    /*
-//     * Input format for edges.
-//     * Edges are separated by new line '\n'.
-//     * An edge is represented as two Integer vertex IDs which are separated by a blank ','.
-//     */
-//    val edges = DataSource(edgeInput, CsvInputFormat[(Int, Int)]("\n", ','))
-//
-//    /*
-//     * Emit each edge twice with both vertex orders.
-//     */
-//    val projEdges = edges flatMap { (e) => Iterator((e._1, e._2) , (e._2, e._1)) }
-//
-//    /*
-//     * Annotates each edges with degree for the first vertex.
-//     */
-//    val vertexCnts = projEdges groupBy { _._1 } reduceGroup { annotateFirstVertexDegree } flatMap {x => x.iterator }
-//
-//    /*
-//     * Combines the degrees of both vertexes of an edge.
-//     */
-//    val combinedVertexCnts = vertexCnts groupBy { (x) => (x._1, x._2) } reduceGroup { combineVertexDegrees }
-//
-//    /*
-//     * Emit annotated edges.
-//     */
-//    val output = combinedVertexCnts.write(annotatedEdgeOutput, DelimitedOutputFormat(formatEdgeWithDegrees.tupled))
-//
-//    val plan = new ScalaPlan(Seq(output), "Compute Edge Degrees")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//
-//  object RunComputeEdgeDegrees {
-//  def main(args: Array[String]) {
-//    val ced = new ComputeEdgeDegrees
-//    if (args.size < 3) {
-//      println(ced.getDescription)
-//      return
-//    }
-//    val plan = ced.getScalaPlan(args(0).toInt, args(1), args(2))
-//    LocalExecutor.execute(plan)
-//    System.exit(0)
-//  }
-//}
-//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
deleted file mode 100644
index 2ef029f..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
+++ /dev/null
@@ -1,108 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.examples.scala.graph;
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-///**
-// * An example program computing the page rank for each vertex in a graph.
-// * The graph is initially represented by vertices and edges. Vertices are numeric identifiers, while
-// * edges are pairs of identifiers that represent the source and target vertex.
-// *
-// * This variant of page rank assumes that all edges that originate at one vertex have an equal
-// * probability of being chosen.
-// */
-//class PageRank extends Program with Serializable {
-//
-//  def getScalaPlan(verticesPath: String, edgesPath: String, outputPath: String, numVertices: Long, maxIterations: Int) = {
-//
-//    case class PageWithRank(pageId: Long, rank: Double)
-//    case class Edge(from: Long, to: Long)
-//    case class Adjacency(vertex: Long, neighbors: List[Long])
-//
-//    // read the pages and edges. the pages are only single decimal identifiers, the edges pairs of identifiers
-//    val pages = DataSource(verticesPath, CsvInputFormat[Long]())
-//    val edges = DataSource(edgesPath, CsvInputFormat[Edge]("\n", ' '))
-//
-//    // some constants used in the specific rank computation
-//    val dampening = 0.85
-//    val randomJump = (1.0 - dampening) / numVertices
-//    val initialRank = 1.0 / numVertices
-//
-//    // assign the initial uniform rank to all pages
-//    val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
-//
-//    // transform the edges from a list of (from -> target) pairs to an adjacency list (from -> [all-targets])
-//    val adjacencies = edges.groupBy(_.from).reduceGroup(x => x.foldLeft(Adjacency(0, List[Long]()))((a, e) => Adjacency(e.from, e.to :: a.neighbors)));
-//
-//    def computeRank(ranks: DataSetOLD[PageWithRank]) = {
-//
-//      val ranksForNeighbors = ranks join adjacencies where { _.pageId } isEqualTo { _.vertex } flatMap ( (p, e) => {
-//        val numNeighbors = e.neighbors.length
-//
-//        for (target <- e.neighbors)
-//          yield (target, p.rank / numNeighbors)
-//
-//      });
-//
-//      ranksForNeighbors.groupBy { case (node, rank) => node }
-//        .reduce { (a, b) => (a._1, a._2 + b._2) }
-//        .map { case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
-//    }
-//
-//    val finalRanks = pagesWithRank.iterate(maxIterations, computeRank)
-//
-//    val output = finalRanks.write(outputPath, CsvOutputFormat())
-//
-//    new ScalaPlan(Seq(output), "Page Rank")
-//  }
-//
-//  override def getPlan(args: String*) = {
-//    val planArgs: Array[String] = if (args.length < 5) Array[String]("", "", "", "", "") else args.toArray
-//    val dop = if (args.size > 5) args(5).toInt else 1
-//
-//    val plan = getScalaPlan(planArgs(0), planArgs(1), planArgs(2), planArgs(3).toLong, planArgs(4).toInt)
-//    plan.setDefaultParallelism(dop)
-//    plan
-//  }
-//}
-//
-///**
-// * Executable entry point to run the program locally.
-// */
-//object RunPageRank {
-//
-//  def main(args: Array[String]) {
-//    if (args.size < 5) {
-//      println("PageRank <pages input path> <links input path> <result path> <num pages> <num iterations> [<parallelism=1>]")
-//      return
-//    }
-//
-//    val dop = if (args.length > 5) args(5).toInt else 1
-//    val plan = new PageRank().getScalaPlan(args(0), args(1), args(2), args(3).toLong, args(4).toInt);
-//
-//    plan.setDefaultParallelism(dop)
-//    LocalExecutor.execute(plan)
-//  }
-//}
-//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
new file mode 100644
index 0000000..e24727c
--- /dev/null
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.graph
+
+import scala.collection.JavaConverters._
+import org.apache.flink.api.scala._
+import org.apache.flink.example.java.graph.util.PageRankData
+import org.apache.flink.util.Collector
+import org.apache.flink.api.java.aggregation.Aggregations.SUM
+
+
+/**
+ * A basic implementation of the Page Rank algorithm using a bulk iteration.
+ * 
+ * <p>
+ * This implementation requires a set of pages and a set of directed links as input and works as follows. <br> 
+ * In each iteration, the rank of every page is evenly distributed to all pages it points to.
+ * Each page collects the partial ranks of all pages that point to it, sums them up, and applies a dampening factor to the sum.
+ * The result is the new rank of the page. A new iteration is started with the new ranks of all pages.
+ * This implementation terminates after a fixed number of iterations.<br>
+ * This is the Wikipedia entry for the <a href="http://en.wikipedia.org/wiki/Page_rank">Page Rank algorithm</a>. 
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Pages represented as an (long) ID separated by new-line characters.<br> 
+ * For example <code>"1\n2\n12\n42\n63\n"</code> gives five pages with IDs 1, 2, 12, 42, and 63.
+ * <li>Links are represented as pairs of page IDs which are separated by space 
+ * characters. Links are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).<br>
+ * For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+ * </ul>
+ * 
+ * <p>
+ * Usage: <code>PageRankBasic &lt;pages path&gt; &lt;links path&gt; &lt;output path&gt; &lt;num pages&gt; &lt;num iterations&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link PageRankData} and 10 iterations.
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Bulk Iterations
+ * <li>Default Join
+ * <li>Configure user-defined functions using constructor parameters.
+ * </ul> 
+ *
+ */
+object PageRankBasic {
+	
+	private final val DAMPENING_FACTOR: Double = 0.85;
+	private final val EPSILON: Double = 0.0001;
+
+	def main(args: Array[String]) {
+		if (!parseParameters(args)) {
+			return
+		}
+		
+		// set up execution environment
+		val env = ExecutionEnvironment.getExecutionEnvironment
+		
+		// read input data
+		val pages = getPagesDataSet(env)
+		val links = getLinksDataSet(env)
+		
+		// assign initial ranks to pages
+		val pagesWithRanks = pages.map(p => Page(p, (1.0/numPages)))
+		
+		// build adjacency list from link input
+		val adjacencyLists = links
+								// initialize lists
+								.map( e => AdjacencyList(e.sourceId, Array[java.lang.Long](e.targetId) ))
+								// concatenate lists
+								.groupBy(0).reduce( (l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds))
+		
+		// start iteration
+		val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) {
+			currentRanks => 
+				val newRanks = currentRanks
+								// distribute ranks to target pages
+								.join(adjacencyLists).where(0).equalTo(0)
+								.flatMap { x => for(targetId <- x._2.targetIds) yield Page(targetId, (x._1.rank / x._2.targetIds.length))}
+								// collect ranks and sum them up
+								.groupBy(0).aggregate(SUM, 1)
+								// apply dampening factor
+								.map { p => Page(p.pageId, (p.rank * DAMPENING_FACTOR) + ((1 - DAMPENING_FACTOR) / numPages) ) }
+				
+				// terminate if no rank update was significant
+				val termination = currentRanks
+									.join(newRanks).where(0).equalTo(0)
+									// check for significant update
+									.filter( x => math.abs(x._1.rank - x._2.rank) > EPSILON )
+				
+				(newRanks, termination)
+		}
+		
+		val result = finalRanks;
+								
+		// emit result
+		if (fileOutput) {
+			result.writeAsCsv(outputPath, "\n", " ")
+		} else {
+			result.print()
+		}
+		
+		// execute program
+		env.execute("Basic PageRank Example")
+	}
+	
+	// *************************************************************************
+	//     USER TYPES
+	// *************************************************************************
+
+	case class Link(sourceId: Long, targetId: Long)
+	case class Page(pageId: java.lang.Long, rank: Double)
+	case class AdjacencyList(sourceId: java.lang.Long, targetIds: Array[java.lang.Long])
+	
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+
+	private def parseParameters(args: Array[String]): Boolean = {
+		if (args.length > 0) {
+			fileOutput = true
+			if (args.length == 5) {
+				pagesInputPath = args(0)
+				linksInputPath = args(1)
+				outputPath = args(2)
+				numPages = args(3).toLong
+				maxIterations = args(4).toInt
+			} else {
+				System.err.println("Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
+				false
+			}
+		} else {
+			System.out.println("Executing PageRank Basic example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: PageRankBasic <pages path> <links path> <output path> <num pages> <num iterations>");
+			
+			numPages = PageRankData.getNumberOfPages();
+		}
+		true
+	}
+
+	private def getPagesDataSet(env: ExecutionEnvironment): DataSet[Long] = {
+		if(fileOutput) {
+			env.readCsvFile[Tuple1[Long]](pagesInputPath, fieldDelimiter = ' ', lineDelimiter = "\n")
+				.map(x => x._1)
+		} else {
+			env.fromCollection(Seq.range(1, PageRankData.getNumberOfPages()+1))
+		}
+	}
+	
+	private def getLinksDataSet(env: ExecutionEnvironment): DataSet[Link] = {
+		if (fileOutput) {
+			env.readCsvFile[(Long, Long)](linksInputPath, fieldDelimiter = ' ', includedFields = Array(0, 1))
+				.map { x => Link(x._1, x._2) }
+		} else {
+			val edges = PageRankData.EDGES.map{ case Array(v1, v2) => Link(v1.asInstanceOf[Long], v2.asInstanceOf[Long]) }
+			env.fromCollection(edges)
+		}
+	}	
+	
+	private var fileOutput: Boolean = false
+	private var pagesInputPath: String = null
+	private var linksInputPath: String = null
+	private var outputPath: String = null
+	private var numPages: Long = 0;
+	private var maxIterations: Int = 10;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
deleted file mode 100644
index 97ee62e..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
+++ /dev/null
@@ -1,108 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.examples.scala.graph;
-//
-//import scala.math._
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//import org.apache.flink.api.scala.analysis.GlobalSchemaPrinter
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//import org.apache.flink.api.common.Plan
-//import org.apache.flink.api.java.record.operators.DeltaIteration
-//
-///**
-// * An implementation of the PageRank algorithm for graph vertex ranking. Runs a specified fix number
-// * of iterations. This version of page rank expects the edges to define a transition
-// * probability and hence allows to model situations where not all outgoing links are equally probable.
-// *
-// * <p>
-// *
-// * Expects inputs are:
-// *  1. Path to a file of node ids, as a sequence of Longs, line delimited.
-// *  2. Path to a csv file of edges in the format <tt>sourceId targetId transitionProbability</tt> (fields separated by spaces).
-// *    The ids are expected to be Longs, the transition probability a float or double.
-// *  3. Path to where the output should be written
-// *  4. The number of vertices
-// *  5. The number of iterations
-// */
-//class PageRankWithWeight extends Program with Serializable {
-//
-//  def getScalaPlan(verticesPath: String, edgesPath: String, outputPath: String, numVertices: Long, maxIterations: Int) = {
-//
-//    case class PageWithRank(pageId: Long, rank: Double)
-//    case class Edge(from: Long, to: Long, transitionProb: Double)
-//
-//    val pages = DataSource(verticesPath, CsvInputFormat[Long]())
-//    val edges = DataSource(edgesPath, CsvInputFormat[Edge]("\n", ' ')) // line delimiter (\n), field delimiter (' ')
-//
-//    val dampening = 0.85
-//    val randomJump = (1.0 - dampening) / numVertices
-//    val initialRank = 1.0 / numVertices
-//
-//    val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
-//
-//    def computeRank(ranks: DataSetOLD[PageWithRank]) = {
-//
-//      val ranksForNeighbors = ranks join edges where { _.pageId } isEqualTo { _.from } map { (p, e) => (e.to, p.rank * e.transitionProb) }
-//
-//      ranksForNeighbors.groupBy { case (node, rank) => node }
-//        .reduce { (a, b) => (a._1, a._2 + b._2) }
-//        .map { case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
-//    }
-//
-//    val finalRanks = pagesWithRank.iterate(maxIterations, computeRank)
-//
-//    val output = finalRanks.write(outputPath, CsvOutputFormat())
-//
-//    new ScalaPlan(Seq(output), "Connected Components")
-//  }
-//
-//  override def getPlan(args: String*) = {
-//    val planArgs: Array[String] = if (args.length < 5) Array[String]("", "", "", "", "") else args.toArray
-//    val dop = if (args.size > 5) args(5).toInt else 1
-//
-//    val plan = getScalaPlan(planArgs(0), planArgs(1), planArgs(2), planArgs(3).toLong, planArgs(4).toInt)
-//    plan.setDefaultParallelism(dop)
-//    plan
-//  }
-//}
-//
-///**
-// * Executable entry point to run the program locally.
-// */
-//object RunPageRankWithWeight {
-//
-//  def main(args: Array[String]) {
-//    if (args.size < 5) {
-//      println("PageRank <vertices> <edges> <result> <numVertices> <numIterations> [<parallelism=1>]")
-//      return
-//    }
-//
-//    val dop = if (args.length > 5) args(5).toInt else 1
-//    val plan = new PageRankWithWeight().getScalaPlan(args(0), args(1), args(2), args(3).toLong, args(4).toInt);
-//
-//    plan.setDefaultParallelism(dop)
-//    LocalExecutor.execute(plan)
-//  }
-//}
-//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0dc76147/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
index 241a376..eb6fe9f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
@@ -23,7 +23,6 @@ import static org.apache.flink.api.java.aggregation.Aggregations.SUM;
 import static org.junit.Assert.fail;
 
 import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.compiler.PactCompiler;
 import org.apache.flink.compiler.plan.BulkIterationPlanNode;
@@ -53,8 +52,7 @@ public class PageRankCompilerTest extends CompilerTestBase{
 			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 			
 			// get input data
-			@SuppressWarnings("unchecked")
-			DataSet<Tuple1<Long>> pagesInput = env.fromElements(new Tuple1<Long>(1l));
+			DataSet<Long> pagesInput = env.fromElements(1l);
 			@SuppressWarnings("unchecked")
 			DataSet<Tuple2<Long, Long>> linksInput =env.fromElements(new Tuple2<Long, Long>(1l, 2l));
 			


[45/60] git commit: Rename ScalaTupleTypeInfo to CaseClassTypeInfo

Posted by al...@apache.org.
Rename ScalaTupleTypeInfo to CaseClassTypeInfo

This better reflects what it is actually for. It is still derived
from TupleTypeInfoBase, though.


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

Branch: refs/heads/master
Commit: ca4fa3da0f32dd6c07b84768fe8d089d9cb8c482
Parents: 0385651
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 16 14:15:06 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/api/scala/DataSet.scala    |   2 +-
 .../apache/flink/api/scala/GroupedDataSet.scala |   2 +-
 .../apache/flink/api/scala/coGroupDataSet.scala |   6 +-
 .../api/scala/codegen/TypeInformationGen.scala  |   6 +-
 .../apache/flink/api/scala/crossDataSet.scala   |   6 +-
 .../apache/flink/api/scala/joinDataSet.scala    |   6 +-
 .../org/apache/flink/api/scala/package.scala    |   4 +-
 .../scala/typeutils/CaseClassComparator.scala   | 148 +++++++++++++++++++
 .../scala/typeutils/CaseClassSerializer.scala   |  65 ++++++++
 .../api/scala/typeutils/CaseClassTypeInfo.scala |  88 +++++++++++
 .../scala/typeutils/ScalaTupleComparator.scala  | 148 -------------------
 .../scala/typeutils/ScalaTupleSerializer.scala  |  65 --------
 .../scala/typeutils/ScalaTupleTypeInfo.scala    |  88 -----------
 .../api/scala/unfinishedKeyPairOperation.scala  |   2 +-
 .../runtime/GenericPairComparatorTest.scala     |   6 +-
 .../tuple/base/TupleComparatorTestBase.scala    |   2 +-
 16 files changed, 322 insertions(+), 322 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 0d0519c..8aabce4 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -29,7 +29,7 @@ import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
 import org.apache.flink.api.java.operators._
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
 import org.apache.flink.api.scala.operators.{ScalaCsvOutputFormat, ScalaAggregateOperator}
-import org.apache.flink.api.scala.typeutils.ScalaTupleTypeInfo
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.core.fs.{FileSystem, Path}
 import org.apache.flink.types.TypeInformation

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
index 3c5bf9e..88a8c7c 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
@@ -19,7 +19,7 @@ package org.apache.flink.api.scala
 
 import org.apache.flink.api.common.InvalidProgramException
 import org.apache.flink.api.scala.operators.ScalaAggregateOperator
-import org.apache.flink.api.scala.typeutils.ScalaTupleTypeInfo
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 
 import scala.collection.JavaConverters._
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
index 582edac..7613c54 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer
 import org.apache.flink.api.java.operators._
 import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
-import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo}
 import org.apache.flink.types.TypeInformation
 import org.apache.flink.util.Collector
 
@@ -177,7 +177,7 @@ private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
     val rightArrayType =
       ObjectArrayTypeInfo.getInfoFor(new Array[O](0).getClass, rightSet.set.getType)
 
-    val returnType = new ScalaTupleTypeInfo[(Array[T], Array[O])](
+    val returnType = new CaseClassTypeInfo[(Array[T], Array[O])](
       classOf[(Array[T], Array[O])], Seq(leftArrayType, rightArrayType), Array("_1", "_2")) {
 
       override def createSerializer: TypeSerializer[(Array[T], Array[O])] = {
@@ -186,7 +186,7 @@ private[flink] class UnfinishedCoGroupOperationImpl[T: ClassTag, O: ClassTag](
           fieldSerializers(i) = types(i).createSerializer
         }
 
-        new ScalaTupleSerializer[(Array[T], Array[O])](
+        new CaseClassSerializer[(Array[T], Array[O])](
           classOf[(Array[T], Array[O])],
           fieldSerializers) {
           override def createInstance(fields: Array[AnyRef]) = {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index 235caa7..cfa5a21 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -19,7 +19,7 @@ package org.apache.flink.api.scala.codegen
 
 import org.apache.flink.api.common.typeutils.TypeSerializer
 import org.apache.flink.api.java.typeutils._
-import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo}
 import org.apache.flink.types.{Value, TypeInformation}
 import org.apache.hadoop.io.Writable
 
@@ -70,14 +70,14 @@ private[flink] trait TypeInformationGen[C <: Context] {
     val fieldNames = desc.getters map { f => Literal(Constant(f.getter.name.toString)) } toList
     val fieldNamesExpr = c.Expr[Seq[String]](mkSeq(fieldNames))
     reify {
-      new ScalaTupleTypeInfo[T](tpeClazz.splice, fieldsExpr.splice, fieldNamesExpr.splice) {
+      new CaseClassTypeInfo[T](tpeClazz.splice, fieldsExpr.splice, fieldNamesExpr.splice) {
         override def createSerializer: TypeSerializer[T] = {
           val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
           for (i <- 0 until getArity) {
             fieldSerializers(i) = types(i).createSerializer
           }
 
-          new ScalaTupleSerializer[T](tupleType, fieldSerializers) {
+          new CaseClassSerializer[T](tupleType, fieldSerializers) {
             override def createInstance(fields: Array[AnyRef]): T = {
               instance.splice
             }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
index df34587..97e48b1 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.{RichCrossFunction, CrossFunction}
 import org.apache.flink.api.common.typeutils.TypeSerializer
 import org.apache.flink.api.java.operators._
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
-import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo}
 import org.apache.flink.types.TypeInformation
 import org.apache.flink.util.Collector
 
@@ -109,7 +109,7 @@ private[flink] object CrossDataSetImpl {
         (left, right)
       }
     }
-    val returnType = new ScalaTupleTypeInfo[(T, O)](
+    val returnType = new CaseClassTypeInfo[(T, O)](
       classOf[(T, O)], Seq(leftSet.getType, rightSet.getType), Array("_1", "_2")) {
 
       override def createSerializer: TypeSerializer[(T, O)] = {
@@ -118,7 +118,7 @@ private[flink] object CrossDataSetImpl {
           fieldSerializers(i) = types(i).createSerializer
         }
 
-        new ScalaTupleSerializer[(T, O)](classOf[(T, O)], fieldSerializers) {
+        new CaseClassSerializer[(T, O)](classOf[(T, O)], fieldSerializers) {
           override def createInstance(fields: Array[AnyRef]) = {
             (fields(0).asInstanceOf[T], fields(1).asInstanceOf[O])
           }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index a607301..fa63c4f 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -25,7 +25,7 @@ import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin.WrappingFlat
 import org.apache.flink.api.java.operators.JoinOperator.{EquiJoin, JoinHint}
 import org.apache.flink.api.java.operators._
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
-import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo}
 import org.apache.flink.types.TypeInformation
 import org.apache.flink.util.Collector
 
@@ -179,7 +179,7 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
         out.collect((left, right))
       }
     }
-    val returnType = new ScalaTupleTypeInfo[(T, O)](
+    val returnType = new CaseClassTypeInfo[(T, O)](
       classOf[(T, O)], Seq(leftSet.set.getType, rightSet.set.getType), Array("_1", "_2")) {
 
       override def createSerializer: TypeSerializer[(T, O)] = {
@@ -188,7 +188,7 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
           fieldSerializers(i) = types(i).createSerializer
         }
 
-        new ScalaTupleSerializer[(T, O)](classOf[(T, O)], fieldSerializers) {
+        new CaseClassSerializer[(T, O)](classOf[(T, O)], fieldSerializers) {
           override def createInstance(fields: Array[AnyRef]) = {
             (fields(0).asInstanceOf[T], fields(1).asInstanceOf[O])
           }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
index 73dd721..8bb69f9 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -21,7 +21,7 @@ package org.apache.flink.api
 import _root_.scala.reflect.ClassTag
 import language.experimental.macros
 import org.apache.flink.types.TypeInformation
-import org.apache.flink.api.scala.typeutils.{ScalaTupleTypeInfo, TypeUtils}
+import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
 
 package object scala {
@@ -36,7 +36,7 @@ package object scala {
       typeInfo: TypeInformation[_],
       fields: Array[String]): Array[Int] = {
     typeInfo match {
-      case ti: ScalaTupleTypeInfo[_] =>
+      case ti: CaseClassTypeInfo[_] =>
         ti.getFieldIndices(fields)
 
       case _ =>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassComparator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassComparator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassComparator.scala
new file mode 100644
index 0000000..ae7892f
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassComparator.scala
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.api.java.typeutils.runtime.TupleComparatorBase
+import org.apache.flink.core.memory.MemorySegment
+import org.apache.flink.types.{KeyFieldOutOfBoundsException, NullKeyFieldException}
+;
+
+/**
+ * Comparator for Case Classes. Access is different from
+ * our Java Tuples so we have to treat them differently.
+ */
+class CaseClassComparator[T <: Product](
+    keys: Array[Int],
+    scalaComparators: Array[TypeComparator[_]],
+    scalaSerializers: Array[TypeSerializer[_]] )
+  extends TupleComparatorBase[T](keys, scalaComparators, scalaSerializers) {
+
+  private val extractedKeys = new Array[AnyRef](keys.length)
+
+  // We cannot use the Clone Constructor from Scala so we have to do it manually
+  def duplicate: TypeComparator[T] = {
+    // ensure that the serializers are available
+    instantiateDeserializationUtils()
+    val result = new CaseClassComparator[T](keyPositions, comparators, serializers)
+    result.privateDuplicate(this)
+    result
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Comparator Methods
+  // --------------------------------------------------------------------------------------------
+
+  def hash(value: T): Int = {
+    val comparator = comparators(0).asInstanceOf[TypeComparator[Any]]
+    var code: Int = comparator.hash(value.productElement(keyPositions(0)))
+    for (i <- 1 until keyPositions.length) {
+      try {
+        code *= TupleComparatorBase.HASH_SALT(i & 0x1F)
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        code += comparator.hash(value.productElement(keyPositions(i)))
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+    code
+  }
+
+  def setReference(toCompare: T) {
+    for (i <- 0 until keyPositions.length) {
+      try {
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        comparator.setReference(toCompare.productElement(keyPositions(i)))
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+  }
+
+  def equalToReference(candidate: T): Boolean = {
+    for (i <- 0 until keyPositions.length) {
+      try {
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        if (!comparator.equalToReference(candidate.productElement(keyPositions(i)))) {
+          return false
+        }
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+    true
+  }
+
+  def compare(first: T, second: T): Int = {
+    for (i <- 0 until keyPositions.length) {
+      try {
+        val keyPos: Int = keyPositions(i)
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        val cmp: Int = comparator.compare(
+          first.productElement(keyPos),
+          second.productElement(keyPos))
+        if (cmp != 0) {
+          return cmp
+        }
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+    0
+  }
+
+  def putNormalizedKey(value: T, target: MemorySegment, offsetParam: Int, numBytesParam: Int) {
+    var numBytes = numBytesParam
+    var offset = offsetParam
+    var i: Int = 0
+    try {
+      while (i < numLeadingNormalizableKeys && numBytes > 0) {
+        {
+          var len: Int = normalizedKeyLengths(i)
+          len = if (numBytes >= len) len else numBytes
+          val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+          comparator.putNormalizedKey(value.productElement(keyPositions(i)), target, offset, len)
+          numBytes -= len
+          offset += len
+        }
+        i += 1
+      }
+    } catch {
+      case npex: NullPointerException => throw new NullKeyFieldException(keyPositions(i))
+    }
+  }
+
+  def extractKeys(value: T) = {
+    for (i <- 0 until keyPositions.length ) {
+      extractedKeys(i) = value.productElement(keyPositions(i)).asInstanceOf[AnyRef]
+    }
+    extractedKeys
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
new file mode 100644
index 0000000..f910b10
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase
+import org.apache.flink.core.memory.{DataOutputView, DataInputView}
+;
+
+/**
+ * Serializer for Case Classes. Creation and access is different from
+ * our Java Tuples so we have to treat them differently.
+ */
+abstract class CaseClassSerializer[T <: Product](
+    clazz: Class[T],
+    scalaFieldSerializers: Array[TypeSerializer[_]])
+  extends TupleSerializerBase[T](clazz, scalaFieldSerializers) {
+
+  def createInstance: T = {
+    val fields: Array[AnyRef] = new Array(arity)
+    for (i <- 0 until arity) {
+      fields(i) = fieldSerializers(i).createInstance()
+    }
+    createInstance(fields)
+  }
+
+  def copy(from: T, reuse: T): T = {
+    val fields: Array[AnyRef] = new Array(arity)
+    for (i <- 0 until arity) {
+      fields(i) = from.productElement(i).asInstanceOf[AnyRef]
+    }
+    createInstance(fields)
+  }
+
+  def serialize(value: T, target: DataOutputView) {
+    for (i <- 0 until arity) {
+      val serializer = fieldSerializers(i).asInstanceOf[TypeSerializer[Any]]
+      serializer.serialize(value.productElement(i), target)
+    }
+  }
+
+  def deserialize(reuse: T, source: DataInputView): T = {
+    val fields: Array[AnyRef] = new Array(arity)
+    for (i <- 0 until arity) {
+      val field = reuse.productElement(i).asInstanceOf[AnyRef]
+      fields(i) = fieldSerializers(i).deserialize(field, source)
+    }
+    createInstance(fields)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
new file mode 100644
index 0000000..a5e7793
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.java.typeutils.{AtomicType, TupleTypeInfoBase}
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+
+/**
+ * TypeInformation for Case Classes. Creation and access is different from
+ * our Java Tuples so we have to treat them differently.
+ */
+abstract class CaseClassTypeInfo[T <: Product](
+    clazz: Class[T],
+    fieldTypes: Seq[TypeInformation[_]],
+    val fieldNames: Seq[String])
+  extends TupleTypeInfoBase[T](clazz, fieldTypes: _*) {
+
+  def createComparator(logicalKeyFields: Array[Int], orders: Array[Boolean]): TypeComparator[T] = {
+    // sanity checks
+    if (logicalKeyFields == null || orders == null
+      || logicalKeyFields.length != orders.length || logicalKeyFields.length > types.length) {
+      throw new IllegalArgumentException
+    }
+
+    // No special handling of leading Key field as in JavaTupleComparator for now
+
+    // --- general case ---
+    var maxKey: Int = -1
+
+    for (key <- logicalKeyFields) {
+      maxKey = Math.max(key, maxKey)
+    }
+
+    if (maxKey >= types.length) {
+      throw new IllegalArgumentException("The key position " + maxKey + " is out of range for " +
+        "Tuple" + types.length)
+    }
+
+    // create the comparators for the individual fields
+    val fieldComparators: Array[TypeComparator[_]] = new Array(logicalKeyFields.length)
+
+    for (i <- 0 until logicalKeyFields.length) {
+      val keyPos = logicalKeyFields(i)
+      if (types(keyPos).isKeyType && types(keyPos).isInstanceOf[AtomicType[_]]) {
+        fieldComparators(i) = types(keyPos).asInstanceOf[AtomicType[_]].createComparator(orders(i))
+      } else {
+        throw new IllegalArgumentException(
+          "The field at position " + i + " (" + types(keyPos) + ") is no atomic key type.")
+      }
+    }
+
+    // create the serializers for the prefix up to highest key position
+    val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](maxKey + 1)
+
+    for (i <- 0 to maxKey) {
+      fieldSerializers(i) = types(i).createSerializer
+    }
+
+    new CaseClassComparator[T](logicalKeyFields, fieldComparators, fieldSerializers)
+  }
+
+  def getFieldIndices(fields: Array[String]): Array[Int] = {
+    val result = fields map { x => fieldNames.indexOf(x) }
+    if (result.contains(-1)) {
+      throw new IllegalArgumentException("Fields '" + fields.mkString(", ") +
+        "' are not valid for " + clazz + " with fields '" + fieldNames.mkString(", ") + "'.")
+    }
+    result
+  }
+
+  override def toString = "Scala " + super.toString
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
deleted file mode 100644
index 1c8f8df..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.scala.typeutils
-
-import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
-import org.apache.flink.api.java.typeutils.runtime.TupleComparatorBase
-import org.apache.flink.core.memory.MemorySegment
-import org.apache.flink.types.{KeyFieldOutOfBoundsException, NullKeyFieldException}
-;
-
-/**
- * Comparator for Scala Tuples. Access is different from
- * our Java Tuples so we have to treat them differently.
- */
-class ScalaTupleComparator[T <: Product](
-    keys: Array[Int],
-    scalaComparators: Array[TypeComparator[_]],
-    scalaSerializers: Array[TypeSerializer[_]] )
-  extends TupleComparatorBase[T](keys, scalaComparators, scalaSerializers) {
-
-  private val extractedKeys = new Array[AnyRef](keys.length)
-
-  // We cannot use the Clone Constructor from Scala so we have to do it manually
-  def duplicate: TypeComparator[T] = {
-    // ensure that the serializers are available
-    instantiateDeserializationUtils()
-    val result = new ScalaTupleComparator[T](keyPositions, comparators, serializers)
-    result.privateDuplicate(this)
-    result
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Comparator Methods
-  // --------------------------------------------------------------------------------------------
-
-  def hash(value: T): Int = {
-    val comparator = comparators(0).asInstanceOf[TypeComparator[Any]]
-    var code: Int = comparator.hash(value.productElement(keyPositions(0)))
-    for (i <- 1 until keyPositions.length) {
-      try {
-        code *= TupleComparatorBase.HASH_SALT(i & 0x1F)
-        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
-        code += comparator.hash(value.productElement(keyPositions(i)))
-      } catch {
-        case npex: NullPointerException =>
-          throw new NullKeyFieldException(keyPositions(i))
-        case iobex: IndexOutOfBoundsException =>
-          throw new KeyFieldOutOfBoundsException(keyPositions(i))
-      }
-    }
-    code
-  }
-
-  def setReference(toCompare: T) {
-    for (i <- 0 until keyPositions.length) {
-      try {
-        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
-        comparator.setReference(toCompare.productElement(keyPositions(i)))
-      } catch {
-        case npex: NullPointerException =>
-          throw new NullKeyFieldException(keyPositions(i))
-        case iobex: IndexOutOfBoundsException =>
-          throw new KeyFieldOutOfBoundsException(keyPositions(i))
-      }
-    }
-  }
-
-  def equalToReference(candidate: T): Boolean = {
-    for (i <- 0 until keyPositions.length) {
-      try {
-        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
-        if (!comparator.equalToReference(candidate.productElement(keyPositions(i)))) {
-          return false
-        }
-      } catch {
-        case npex: NullPointerException =>
-          throw new NullKeyFieldException(keyPositions(i))
-        case iobex: IndexOutOfBoundsException =>
-          throw new KeyFieldOutOfBoundsException(keyPositions(i))
-      }
-    }
-    true
-  }
-
-  def compare(first: T, second: T): Int = {
-    for (i <- 0 until keyPositions.length) {
-      try {
-        val keyPos: Int = keyPositions(i)
-        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
-        val cmp: Int = comparator.compare(
-          first.productElement(keyPos),
-          second.productElement(keyPos))
-        if (cmp != 0) {
-          return cmp
-        }
-      } catch {
-        case npex: NullPointerException =>
-          throw new NullKeyFieldException(keyPositions(i))
-        case iobex: IndexOutOfBoundsException =>
-          throw new KeyFieldOutOfBoundsException(keyPositions(i))
-      }
-    }
-    0
-  }
-
-  def putNormalizedKey(value: T, target: MemorySegment, offsetParam: Int, numBytesParam: Int) {
-    var numBytes = numBytesParam
-    var offset = offsetParam
-    var i: Int = 0
-    try {
-      while (i < numLeadingNormalizableKeys && numBytes > 0) {
-        {
-          var len: Int = normalizedKeyLengths(i)
-          len = if (numBytes >= len) len else numBytes
-          val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
-          comparator.putNormalizedKey(value.productElement(keyPositions(i)), target, offset, len)
-          numBytes -= len
-          offset += len
-        }
-        i += 1
-      }
-    } catch {
-      case npex: NullPointerException => throw new NullKeyFieldException(keyPositions(i))
-    }
-  }
-
-  def extractKeys(value: T) = {
-    for (i <- 0 until keyPositions.length ) {
-      extractedKeys(i) = value.productElement(keyPositions(i)).asInstanceOf[AnyRef]
-    }
-    extractedKeys
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
deleted file mode 100644
index 90d3b5b..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.scala.typeutils
-
-import org.apache.flink.api.common.typeutils.TypeSerializer
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase
-import org.apache.flink.core.memory.{DataOutputView, DataInputView}
-;
-
-/**
- * Serializer for Scala Tuples. Creation and access is different from
- * our Java Tuples so we have to treat them differently.
- */
-abstract class ScalaTupleSerializer[T <: Product](
-    tupleClass: Class[T],
-    scalaFieldSerializers: Array[TypeSerializer[_]])
-  extends TupleSerializerBase[T](tupleClass, scalaFieldSerializers) {
-
-  def createInstance: T = {
-    val fields: Array[AnyRef] = new Array(arity)
-    for (i <- 0 until arity) {
-      fields(i) = fieldSerializers(i).createInstance()
-    }
-    createInstance(fields)
-  }
-
-  def copy(from: T, reuse: T): T = {
-    val fields: Array[AnyRef] = new Array(arity)
-    for (i <- 0 until arity) {
-      fields(i) = from.productElement(i).asInstanceOf[AnyRef]
-    }
-    createInstance(fields)
-  }
-
-  def serialize(value: T, target: DataOutputView) {
-    for (i <- 0 until arity) {
-      val serializer = fieldSerializers(i).asInstanceOf[TypeSerializer[Any]]
-      serializer.serialize(value.productElement(i), target)
-    }
-  }
-
-  def deserialize(reuse: T, source: DataInputView): T = {
-    val fields: Array[AnyRef] = new Array(arity)
-    for (i <- 0 until arity) {
-      val field = reuse.productElement(i).asInstanceOf[AnyRef]
-      fields(i) = fieldSerializers(i).deserialize(field, source)
-    }
-    createInstance(fields)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
deleted file mode 100644
index ad407cb..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.api.scala.typeutils
-
-import org.apache.flink.api.java.typeutils.{AtomicType, TupleTypeInfoBase}
-import org.apache.flink.types.TypeInformation
-import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
-
-/**
- * TypeInformation for Scala Tuples. Creation and access is different from
- * our Java Tuples so we have to treat them differently.
- */
-abstract class ScalaTupleTypeInfo[T <: Product](
-    tupleClass: Class[T],
-    fieldTypes: Seq[TypeInformation[_]],
-    val fieldNames: Seq[String])
-  extends TupleTypeInfoBase[T](tupleClass, fieldTypes: _*) {
-
-  def createComparator(logicalKeyFields: Array[Int], orders: Array[Boolean]): TypeComparator[T] = {
-    // sanity checks
-    if (logicalKeyFields == null || orders == null
-      || logicalKeyFields.length != orders.length || logicalKeyFields.length > types.length) {
-      throw new IllegalArgumentException
-    }
-
-    // No special handling of leading Key field as in JavaTupleComparator for now
-
-    // --- general case ---
-    var maxKey: Int = -1
-
-    for (key <- logicalKeyFields) {
-      maxKey = Math.max(key, maxKey)
-    }
-
-    if (maxKey >= types.length) {
-      throw new IllegalArgumentException("The key position " + maxKey + " is out of range for " +
-        "Tuple" + types.length)
-    }
-
-    // create the comparators for the individual fields
-    val fieldComparators: Array[TypeComparator[_]] = new Array(logicalKeyFields.length)
-
-    for (i <- 0 until logicalKeyFields.length) {
-      val keyPos = logicalKeyFields(i)
-      if (types(keyPos).isKeyType && types(keyPos).isInstanceOf[AtomicType[_]]) {
-        fieldComparators(i) = types(keyPos).asInstanceOf[AtomicType[_]].createComparator(orders(i))
-      } else {
-        throw new IllegalArgumentException(
-          "The field at position " + i + " (" + types(keyPos) + ") is no atomic key type.")
-      }
-    }
-
-    // create the serializers for the prefix up to highest key position
-    val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](maxKey + 1)
-
-    for (i <- 0 to maxKey) {
-      fieldSerializers(i) = types(i).createSerializer
-    }
-
-    new ScalaTupleComparator[T](logicalKeyFields, fieldComparators, fieldSerializers)
-  }
-
-  def getFieldIndices(fields: Array[String]): Array[Int] = {
-    val result = fields map { x => fieldNames.indexOf(x) }
-    if (result.contains(-1)) {
-      throw new IllegalArgumentException("Fields '" + fields.mkString(", ") +
-        "' are not valid for " + tupleClass + " with fields '" + fieldNames.mkString(", ") + "'.")
-    }
-    result
-  }
-
-  override def toString = "Scala " + super.toString
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
index 8541481..3bf3db2 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
@@ -24,7 +24,7 @@ import org.apache.flink.api.java.{DataSet => JavaDataSet}
 import org.apache.flink.api.java.functions.KeySelector
 import org.apache.flink.api.java.operators.Keys
 import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
-import org.apache.flink.api.scala.typeutils.ScalaTupleTypeInfo
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.types.TypeInformation
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
index 95a1b77..c130b7e 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.base.{DoubleComparator, DoubleSeria
 
 import org.apache.flink.api.java.typeutils.runtime.{GenericPairComparator, TupleComparator}
 import org.apache.flink.api.scala.runtime.tuple.base.PairComparatorTestBase
-import org.apache.flink.api.scala.typeutils.ScalaTupleComparator
+import org.apache.flink.api.scala.typeutils.CaseClassComparator
 
 class GenericPairComparatorTest
   extends PairComparatorTestBase[(Int, String, Double), (Int, Float, Long, Double)] {
@@ -42,8 +42,8 @@ class GenericPairComparatorTest
     val sers2 =
       Array[TypeSerializer[_]](IntSerializer.INSTANCE, DoubleSerializer.INSTANCE)
 
-    val comp1 = new ScalaTupleComparator[(Int, String, Double)](fields1, comps1, sers1)
-    val comp2 = new ScalaTupleComparator[(Int, Float, Long, Double)](fields2, comps2, sers2)
+    val comp1 = new CaseClassComparator[(Int, String, Double)](fields1, comps1, sers1)
+    val comp2 = new CaseClassComparator[(Int, Float, Long, Double)](fields2, comps2, sers2)
 
     new GenericPairComparator[(Int, String, Double), (Int, Float, Long, Double)](comp1, comp2)
   }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ca4fa3da/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
index 5370e61..a142384 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/tuple/base/TupleComparatorTestBase.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.api.scala.runtime.tuple.base
 
 import org.apache.flink.api.common.typeutils.ComparatorTestBase
-import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleComparator}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassComparator}
 import org.junit.Assert._
 
 abstract class TupleComparatorTestBase[T <: Product] extends ComparatorTestBase[T] {


[29/60] git commit: Added TriangleEnumeration examples for reworked Scala API.

Posted by al...@apache.org.
Added TriangleEnumeration examples for reworked Scala API.


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

Branch: refs/heads/master
Commit: dd9e27e98ca4428187e235e7f335ec0393dcbb8a
Parents: e0f2440
Author: Fabian Hueske <fh...@apache.org>
Authored: Sat Sep 6 00:59:44 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../java/graph/util/EnumTrianglesData.java      |  34 ++-
 .../scala/graph/EnumTrianglesBasic.scala        | 179 ++++++++++++++
 .../graph/EnumTrianglesOnEdgesWithDegrees.scala | 125 ----------
 .../examples/scala/graph/EnumTrianglesOpt.scala | 235 +++++++++++++++++++
 4 files changed, 437 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dd9e27e9/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
index 87d998b..02c0531 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/EnumTrianglesData.java
@@ -19,6 +19,9 @@
 
 package org.apache.flink.example.java.graph.util;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Edge;
@@ -30,18 +33,27 @@ import org.apache.flink.example.java.graph.util.EnumTrianglesDataTypes.Edge;
  */
 public class EnumTrianglesData {
 
+	public static final Object[][] EDGES = {
+		{1, 2},
+		{1, 3},
+		{1 ,4},
+		{1, 5},
+		{2, 3},
+		{2, 5},
+		{3, 4},
+		{3, 7},
+		{3, 8},
+		{5, 6},
+		{7, 8}
+	};
+	
 	public static DataSet<Edge> getDefaultEdgeDataSet(ExecutionEnvironment env) {
 		
-		return env.fromElements(new Edge(1, 2),
-								new Edge(1, 3),
-								new Edge(1, 4),
-								new Edge(1, 5),
-								new Edge(2, 3),
-								new Edge(2, 5),
-								new Edge(3, 4),
-								new Edge(3, 7),
-								new Edge(3, 8),
-								new Edge(5, 6),
-								new Edge(7, 8));
+		List<Edge> edges = new ArrayList<Edge>();
+		for(Object[] e : EDGES) {
+			edges.add(new Edge((Integer)e[0], (Integer)e[1]));
+		}
+		
+		return env.fromCollection(edges);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dd9e27e9/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
new file mode 100644
index 0000000..8672b2c
--- /dev/null
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.graph
+
+import org.apache.flink.api.scala._
+import scala.collection.JavaConverters._
+import org.apache.flink.api.scala.ExecutionEnvironment
+import org.apache.flink.api.common.functions.GroupReduceFunction
+import org.apache.flink.util.Collector
+import org.apache.flink.example.java.graph.util.EnumTrianglesData
+import org.apache.flink.api.common.operators.Order
+import scala.collection.mutable.MutableList
+
+
+/**
+ * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
+ * A triangle consists of three edges that connect three vertices with each other.
+ * 
+ * <p>
+ * The algorithm works as follows: 
+ * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
+ * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
+ * that closes the triangle.
+ *  
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Edges are represented as pairs for vertex IDs which are separated by space 
+ * characters. Edges are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
+ * that include a triangle
+ * </ul>
+ * <pre>
+ *     (1)
+ *     /  \
+ *   (2)-(12)
+ * </pre>
+ * 
+ * Usage: 
+ * {{{
+ * EnumTriangleBasic <edge path> <result path>
+ * }}}
+ * <br>
+ * If no parameters are provided, the program is run with default data from 
+ * [[org.apache.flink.example.java.graph.util.EnumTrianglesData]]
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Custom Java objects which extend Tuple
+ * <li>Group Sorting
+ * </ul>
+ * 
+ */
+object EnumTrianglesBasic {
+	
+	def main(args: Array[String]) {
+		if (!parseParameters(args)) {
+			return
+		}
+
+		// set up execution environment
+		val env = ExecutionEnvironment.getExecutionEnvironment
+
+		// read input data
+		val edges = getEdgeDataSet(env)
+		
+		// project edges by vertex id
+		val edgesById = edges map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) )
+		
+		val triangles = edgesById
+						// build triads
+						.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new TriadBuilder())
+						// filter triads
+						.join(edgesById).where(1,2).equalTo(0,1) { (t, _) => Some(t) }
+		
+		// emit result
+		if (fileOutput) {
+			triangles.writeAsCsv(outputPath, "\n", " ")
+		} else {
+			triangles.print()
+		}
+		
+		// execute program
+		env.execute("TriangleEnumeration Example")
+	}
+
+	// *************************************************************************
+	//     USER DATA TYPES
+	// *************************************************************************
+
+	case class Edge(v1: Int, v2: Int) extends Serializable
+	case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
+	
+		
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
+	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
+	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
+	 */
+	class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
+
+		val vertices = MutableList[Integer]()
+		
+		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
+			
+			// clear vertex list
+			vertices.clear
+
+			// build and emit triads
+			for(e <- edges.asScala) {
+			
+				// combine vertex with all previously read vertices
+				for(v <- vertices) {
+					out.collect(Triad(e.v1, v, e.v2))
+				}
+				vertices += e.v2
+			}
+		}
+	}
+
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+
+	private def parseParameters(args: Array[String]): Boolean = {
+		if (args.length > 0) {
+			fileOutput = true
+			if (args.length == 2) {
+				edgePath = args(0)
+				outputPath = args(1)
+			} else {
+				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>")
+				false
+			}
+		} else {
+			System.out.println("Executing Enum Triangles Basic example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>");
+		}
+		true
+	}
+
+	private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
+		if (fileOutput) {
+			env.readCsvFile[(Int, Int)](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1)).
+			map { x => new Edge(x._1, x._2) }
+		} else {
+			val edges = EnumTrianglesData.EDGES.map{ case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int]) }
+			env.fromCollection(edges)
+		}
+	}
+	
+	
+	private var fileOutput: Boolean = false
+	private var edgePath: String = null
+	private var outputPath: String = null
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dd9e27e9/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
deleted file mode 100644
index cf943be..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
+++ /dev/null
@@ -1,125 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.graph
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import scala.math._
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//import org.apache.flink.api.scala.analysis.GlobalSchemaPrinter
-//
-//object RunEnumTrianglesOnEdgesWithDegrees {
-//  def main(args: Array[String]) {
-//    val enumTriangles = new EnumTrianglesOnEdgesWithDegrees
-//    if (args.size < 3) {
-//      println(enumTriangles.getDescription)
-//      return
-//    }
-//    val plan = enumTriangles.getScalaPlan(args(0).toInt, args(1), args(2))
-//    LocalExecutor.execute(plan)
-//  }
-//}
-//
-///**
-// * Enumerates all triangles build by three connected vertices in a graph.
-// * The graph is represented as edges (pairs of vertices) with annotated vertex degrees. *
-// */
-//class EnumTrianglesOnEdgesWithDegrees extends Program with ProgramDescription with Serializable {
-//  override def getDescription() = {
-//    "Parameters: [numSubStasks] [input file] [output file]"
-//  }
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1), args(2))
-//  }
-//
-//  /*
-//   * Output formatting function for triangles.
-//   */
-//  def formatTriangle = (v1: Int, v2: Int, v3: Int) => "%d,%d,%d".format(v1, v2, v3)
-//
-//  /*
-//   * Extracts degree information and projects edges such that lower degree vertex comes first.
-//   */
-//  def projectVertexesWithDegrees(e: (String, String)): (Int, Int) = {
-//    val v1 = e._1.split(",")
-//    val v2 = e._2.split(",")
-//    if (v1(1).toInt <= v2(1).toInt)
-//      (v1(0).toInt, v2(0).toInt)
-//    else
-//      (v2(0).toInt, v1(0).toInt)
-//  }
-//
-//  /*
-//   * Joins projected edges on lower vertex id.
-//   * Emits a triad (triangle candidate with one missing edge) for each unique combination of edges.
-//   * Ensures that vertex 2 and 3 are ordered by vertex id.
-//   */
-//  def buildTriads(eI : Iterator[(Int, Int)]): List[(Int, Int, Int)] = {
-//    val eL = eI.toList
-//    for (e1 <- eL;
-//         e2 <- eL
-//         if e1._2 < e2._2) yield
-//      (e1._1, e1._2, e2._2)
-//  }
-//
-//  def getScalaPlan(numSubTasks: Int, edgeInput: String, triangleOutput: String) = {
-//
-//    /*
-//     * Input format for edges with degrees
-//     * Edges are separated by new line '\n'.
-//     * An edge is represented by two vertex IDs with associated vertex degrees.
-//     * The format of an edge is "<vertexID1>,<vertexDegree1>|<vertexID2>,<vertexDegree2>"
-//     */
-//    val vertexesWithDegrees = DataSource(edgeInput, CsvInputFormat[(String, String)]("\n", '|'))
-//
-//    /*
-//     * Project edges such that vertex with lower degree comes first (record position 1) and remove the degrees.
-//     */
-//    val edgesByDegree = vertexesWithDegrees map { projectVertexesWithDegrees }
-//
-//    /*
-//     * Project edges such that vertex with lower ID comes first (record position) and remove degrees.
-//     */
-//    val edgesByID = edgesByDegree map { (x) => if (x._1 < x._2) (x._1, x._2) else (x._2, x._1) }
-//
-//    /*
-//     * Build triads by joining edges on common vertex.
-//     */
-//    val triads = edgesByDegree groupBy { _._1 } reduceGroup { buildTriads } flatMap {x => x.iterator }
-//
-//    /*
-//     * Join triads with projected edges to 'close' triads.
-//     * This filters triads without a closing edge.
-//     */
-//    val triangles = triads join edgesByID where { t => (t._2, t._3) } isEqualTo { e => (e._1, e._2) } map { (t, e) => t }
-//
-//    /*
-//     * Emit triangles
-//     */
-//    val output = triangles.write(triangleOutput, DelimitedOutputFormat(formatTriangle.tupled))
-//
-//    val plan = new ScalaPlan(Seq(output), "Enumerate Triangles on Edges with Degrees")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dd9e27e9/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
new file mode 100644
index 0000000..65c8b3e
--- /dev/null
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.graph
+
+import org.apache.flink.api.scala._
+import scala.collection.JavaConverters._
+import org.apache.flink.api.scala.ExecutionEnvironment
+import org.apache.flink.api.common.functions.GroupReduceFunction
+import org.apache.flink.util.Collector
+import org.apache.flink.example.java.graph.util.EnumTrianglesData
+import org.apache.flink.api.common.operators.Order
+import scala.collection.mutable.MutableList
+
+
+/**
+ * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
+ * A triangle consists of three edges that connect three vertices with each other.
+ * 
+ * <p>
+ * The basic algorithm works as follows: 
+ * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
+ * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
+ * that closes the triangle.
+ * 
+ * <p>
+ * For a group of <i>n</i> edges that share a common vertex, the number of built triads is quadratic <i>((n*(n-1))/2)</i>.
+ * Therefore, an optimization of the algorithm is to group edges on the vertex with the smaller output degree to 
+ * reduce the number of triads. 
+ * This implementation extends the basic algorithm by computing output degrees of edge vertices and 
+ * grouping on edges on the vertex with the smaller degree.
+ * 
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * <li>Edges are represented as pairs for vertex IDs which are separated by space 
+ * characters. Edges are separated by new-line characters.<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
+ * that include a triangle
+ * </ul>
+ * <pre>
+ *     (1)
+ *     /  \
+ *   (2)-(12)
+ * </pre>
+ * 
+ * Usage: <code>EnumTriangleOpt &lt;edge path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link EnumTrianglesData}.
+ * 
+ * <p>
+ * This example shows how to use:
+ * <ul>
+ * <li>Custom Java objects which extend Tuple
+ * <li>Group Sorting
+ * </ul>
+ * 
+ */
+object EnumTrianglesOpt {
+	
+	def main(args: Array[String]) {
+		if (!parseParameters(args)) {
+			return
+		}
+
+		// set up execution environment
+		val env = ExecutionEnvironment.getExecutionEnvironment
+
+		// read input data
+		val edges = getEdgeDataSet(env)
+		
+		val edgesWithDegrees = edges
+								// duplicate and switch edges
+								.flatMap( e => Array(e, Edge(e.v2, e.v1)) )
+								// add degree of first vertex
+								.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new DegreeCounter())
+								// join degrees of vertices
+								.groupBy(0,2).reduce( (e1, e2) => 	if(e1.d2 == 0) 
+																		new EdgeWithDegrees(e1.v1, e1.d1, e1.v2, e2.d2)
+																	else
+																		new EdgeWithDegrees(e1.v1, e2.d1, e1.v2, e1.d2)
+													)
+													
+		// project edges by degrees, vertex with smaller degree comes first
+		val edgesByDegree = edgesWithDegrees
+								.map(e => if (e.d1 < e.d2) Edge(e.v1, e.v2) else Edge(e.v2, e.v1) )
+		// project edges by Id, vertex with smaller Id comes first
+		val edgesById = edgesByDegree
+								.map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) )
+		
+		val triangles = edgesByDegree
+						// build triads
+						.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup(new TriadBuilder())
+						// filter triads
+						.join(edgesById).where(1,2).equalTo(0,1) { (t, _) => Some(t) }
+		
+		// emit result
+		if (fileOutput) {
+			triangles.writeAsCsv(outputPath, "\n", " ")
+		} else {
+			triangles.print()
+		}
+		
+		// execute program
+		env.execute("TriangleEnumeration Example")
+	}
+
+	// *************************************************************************
+	//     USER DATA TYPES
+	// *************************************************************************
+
+	case class Edge(v1: Int, v2: Int) extends Serializable
+	case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
+	case class EdgeWithDegrees(v1: Int, d1: Int, v2: Int, d2: Int) extends Serializable
+	
+		
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Counts the number of edges that share a common vertex.
+	 * Emits one edge for each input edge with a degree annotation for the shared vertex.
+	 * For each emitted edge, the first vertex is the vertex with the smaller id.
+	 */
+	class DegreeCounter extends GroupReduceFunction[Edge, EdgeWithDegrees] {
+		
+		val vertices = MutableList[Integer]()
+		var groupVertex = 0
+		
+		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[EdgeWithDegrees]) = {
+			
+			// empty vertex list
+			vertices.clear
+			
+			// collect all vertices
+			for(e <- edges.asScala) {
+				groupVertex = e.v1
+				if(!vertices.contains(e.v2) && e.v1 != e.v2) {
+					vertices += e.v2
+				}
+			}
+			
+			// count vertices to obtain degree of groupVertex
+			val degree = vertices.length
+			
+			// create and emit edges with degrees
+			for(v <- vertices) {
+				if (v < groupVertex) {
+					out.collect(new EdgeWithDegrees(v, 0, groupVertex, degree))
+				} else {
+					out.collect(new EdgeWithDegrees(groupVertex, degree, v, 0))
+				}
+			}
+		}
+	}
+	
+	/**
+	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
+	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
+	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
+	 */
+	class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
+
+		val vertices = MutableList[Integer]()
+		
+		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
+			
+			// clear vertex list
+			vertices.clear
+
+			// build and emit triads
+			for(e <- edges.asScala) {
+			
+				// combine vertex with all previously read vertices
+				for(v <- vertices) {
+					out.collect(Triad(e.v1, v, e.v2))
+				}
+				vertices += e.v2
+			}
+		}
+	}
+
+	// *************************************************************************
+	//     UTIL METHODS
+	// *************************************************************************
+
+	private def parseParameters(args: Array[String]): Boolean = {
+		if (args.length > 0) {
+			fileOutput = true
+			if (args.length == 2) {
+				edgePath = args(0)
+				outputPath = args(1)
+			} else {
+				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>")
+				false
+			}
+		} else {
+			System.out.println("Executing Enum Triangles Basic example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>");
+		}
+		true
+	}
+
+	private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
+		if (fileOutput) {
+			env.readCsvFile[(Int, Int)](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1)).
+			map { x => new Edge(x._1, x._2) }
+		} else {
+			val edges = EnumTrianglesData.EDGES.map{ case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int]) }
+			env.fromCollection(edges)
+		}
+	}
+	
+	
+	private var fileOutput: Boolean = false
+	private var edgePath: String = null
+	private var outputPath: String = null
+
+}
\ No newline at end of file


[07/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/FieldSelector.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/FieldSelector.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/FieldSelector.scala
deleted file mode 100644
index d201585..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/FieldSelector.scala
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-import FieldSet.toSeq
-
-/**
- * Instances of this class are typically created by the field selector macros fieldSelectorImpl
- * and keySelectorImpl in {@link FieldSelectorMacros}. 
- *
- * In addition to the language restrictions applied to the lambda expression, the selected fields
- * must also be top-level. Nested fields (such as a list element or an inner instance of a
- * recursive type) are not allowed.
- *
- * @param selection The selected fields
- */
-class FieldSelector(udt: UDT[_], selection: List[Int]) extends Serializable {
-  
-  val inputFields = FieldSet.newInputSet(udt.numFields)
-  val selectionIndices = udt.getSelectionIndices(selection)
-  val selectedFields = inputFields.select(selectionIndices)
-
-  for (field <- inputFields.diff(selectedFields))
-    field.isUsed = false
-  
-  def copy() = new FieldSelector(udt, selection)
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaFields.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaFields.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaFields.scala
deleted file mode 100644
index 80cd455..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaFields.scala
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-abstract sealed class Field extends Serializable {
-  val localPos: Int
-  val globalPos: GlobalPos
-  var isUsed: Boolean = true
-}
-
-case class InputField(val localPos: Int, val globalPos: GlobalPos = new GlobalPos) extends Field
-case class OutputField(val localPos: Int, val globalPos: GlobalPos = new GlobalPos) extends Field
-
-class FieldSet[+FieldType <: Field] private (private val fields: Seq[FieldType]) extends Serializable {
-
-  private var globalized: Boolean = false
-  def isGlobalized = globalized
-
-  def setGlobalized(): Unit = {
-    assert(!globalized, "Field set has already been globalized")
-    globalized = true
-  }
-
-  def apply(localPos: Int): FieldType = {
-    fields.find(_.localPos == localPos).get
-  }
-
-  def select(selection: Seq[Int]): FieldSet[FieldType] = {
-    val outer = this
-    new FieldSet[FieldType](selection map apply) {
-      override def setGlobalized() = outer.setGlobalized()
-      override def isGlobalized = outer.isGlobalized
-    }
-  }
-
-  def toSerializerIndexArray: Array[Int] = fields map {
-    case field if field.isUsed => field.localPos
-//    case field if field.isUsed => field.globalPos.getValue
-    case _                     => -1
-  } toArray
-
-  def toIndexSet: Set[Int] = fields.map(_.localPos).toSet
-//  def toIndexSet: Set[Int] = fields.filter(_.isUsed).map(_.globalPos.getValue).toSet
-
-  def toIndexArray: Array[Int] = fields.map { _.localPos }.toArray
-
-//  def mapToArray[T: ClassTag](fun: FieldType => T): Array[T] = {
-//    (fields map fun) toArray
-//  }
-}
-
-object FieldSet {
-
-  def newInputSet(numFields: Int): FieldSet[InputField] = new FieldSet((0 until numFields) map { InputField(_) })
-  def newOutputSet(numFields: Int): FieldSet[OutputField] = new FieldSet((0 until numFields) map { OutputField(_) })
-
-  def newInputSet[T](udt: UDT[T]): FieldSet[InputField] = newInputSet(udt.numFields)
-  def newOutputSet[T](udt: UDT[T]): FieldSet[OutputField] = newOutputSet(udt.numFields)
-
-  implicit def toSeq[FieldType <: Field](fieldSet: FieldSet[FieldType]): Seq[FieldType] = fieldSet.fields
-}
-
-class GlobalPos extends Serializable {
-
-  private var pos: Either[Int, GlobalPos] = null
-
-  def getValue: Int = pos match {
-    case null          => -1
-    case Left(index)   => index
-    case Right(target) => target.getValue
-  }
-
-  def getIndex: Option[Int] = pos match {
-    case null | Right(_) => None
-    case Left(index)     => Some(index)
-  }
-
-  def getReference: Option[GlobalPos] = pos match {
-    case null | Left(_) => None
-    case Right(target)  => Some(target)
-  }
-
-  def resolve: GlobalPos = pos match {
-    case null          => this
-    case Left(_)       => this
-    case Right(target) => target.resolve
-  }
-
-  def isUnknown = pos == null
-  def isIndex = (pos != null) && pos.isLeft
-  def isReference = (pos != null) && pos.isRight
-
-  def setIndex(index: Int) = {
-    assert(pos == null || pos.isLeft, "Cannot convert a position reference to an index")
-    pos = Left(index)
-  }
-
-  def setReference(target: GlobalPos) = {
-//    assert(pos == null, "Cannot overwrite a known position with a reference")
-    pos = Right(target)
-  }
-}
-
-object GlobalPos {
-
-  object Unknown {
-    def unapply(globalPos: GlobalPos): Boolean = globalPos.isUnknown
-  }
-
-  object Index {
-    def unapply(globalPos: GlobalPos): Option[Int] = globalPos.getIndex
-  }
-
-  object Reference {
-    def unapply(globalPos: GlobalPos): Option[GlobalPos] = globalPos.getReference
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaGenerator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaGenerator.scala
deleted file mode 100644
index 609d41a..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaGenerator.scala
+++ /dev/null
@@ -1,178 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-import java.util.{List => JList}
-
-import scala.Some
-
-import org.apache.flink.api.scala._
-
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.java.record.operators._
-import org.apache.flink.api.common.operators.base.{BulkIterationBase => BulkIteration, DeltaIterationBase => DeltaIteration, GenericDataSourceBase, MapOperatorBase}
-import org.apache.flink.api.common.operators.Union
-import org.apache.flink.types.Record
-
-
-class GlobalSchemaGenerator {
-
-  def initGlobalSchema(sinks: Seq[Operator[Record] with ScalaOperator[_, _]]): Unit = {
-    // don't do anything, we don't need global positions if we don't do reordering of operators
-    // FieldSet.toSerializerIndexArray returns local positions and ignores global positions
-    // sinks.foldLeft(0) { (freePos, contract) => globalizeContract(contract, Seq(), Map(), None, freePos) }
-  }
-
-  /**
-   * Computes disjoint write sets for a contract and its inputs.
-   *
-   * @param contract The contract to globalize
-   * @param parentInputs Input fields which should be bound to the contract's outputs
-   * @param proxies Provides contracts for iteration placeholders
-   * @param fixedOutputs Specifies required positions for the contract's output fields, or None to allocate new positions
-   * @param freePos The current first available position in the global schema
-   * @return The new first available position in the global schema
-   */
-  private def globalizeContract(contract: Operator[Record], parentInputs: Seq[FieldSet[InputField]], proxies: Map[Operator[Record], Operator[Record] with ScalaOperator[_, _]], fixedOutputs: Option[FieldSet[Field]], freePos: Int): Int = {
-
-    val contract4s = proxies.getOrElse(contract, contract.asInstanceOf[Operator[Record] with ScalaOperator[_, _]])
-
-    parentInputs.foreach(contract4s.getUDF.attachOutputsToInputs)
-
-    contract4s.getUDF.outputFields.isGlobalized match {
-
-      case true => freePos
-
-      case false => {
-
-        val freePos1 = globalizeContract(contract4s, proxies, fixedOutputs, freePos)
-
-        contract4s.persistConfiguration(None)
-
-        freePos1
-      }
-    }
-  }
-
-  private def globalizeContract(contract: Operator[_] with ScalaOperator[_, _], proxies: Map[Operator[Record], Operator[Record] with ScalaOperator[_, _]], fixedOutputs: Option[FieldSet[Field]], freePos: Int): Int = {
-
-    contract match {
-
-      case contract : FileDataSink with ScalaOutputOperator[_] => {
-        contract.getUDF.outputFields.setGlobalized()
-        globalizeContract(contract.getInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.asInstanceOf[UDF1[_,_]].inputFields), proxies, None, freePos)
-      }
-
-      case contract: GenericDataSourceBase[_, _] with ScalaOperator[_, _] => {
-        contract.getUDF.setOutputGlobalIndexes(freePos, fixedOutputs)
-      }
-
-      case contract : BulkIteration[_] with BulkIterationScalaOperator[_] => {
-        val s0 = contract.getInput().asInstanceOf[Operator[Record]]
-
-        val s0contract = proxies.getOrElse(s0, s0.asInstanceOf[Operator[Record] with ScalaOperator[_, Record]])
-        val newProxies = proxies + (contract.getPartialSolution().asInstanceOf[Operator[Record]] -> s0contract)
-
-        val freePos1 = globalizeContract(s0, Seq(), proxies, fixedOutputs, freePos)
-        val freePos2 = globalizeContract(contract.getNextPartialSolution().asInstanceOf[Operator[Record]], Seq(), newProxies, Some(s0contract.getUDF.outputFields), freePos1)
-        val freePos3 = Option(contract.getTerminationCriterion().asInstanceOf[Operator[Record]]) map { globalizeContract(_, Seq(), newProxies, None, freePos2) } getOrElse freePos2
-
-        contract.getUDF.assignOutputGlobalIndexes(s0contract.getUDF.outputFields)
-
-        freePos3
-      }
-
-      case contract : DeltaIteration[_, _] with DeltaIterationScalaOperator[_] => {
-//      case contract @ WorksetIterate4sContract(s0, ws0, deltaS, newWS, placeholderS, placeholderWS) => {
-        val s0 = contract.getInitialSolutionSet().asInstanceOf[Operator[Record]]
-        val ws0 = contract.getInitialWorkset().asInstanceOf[Operator[Record]]
-        val deltaS = contract.getSolutionSetDelta.asInstanceOf[Operator[Record]]
-        val newWS = contract.getNextWorkset.asInstanceOf[Operator[Record]]
-
-        val s0contract = proxies.getOrElse(s0, s0.asInstanceOf[Operator[Record] with ScalaOperator[_, _]])
-        val ws0contract = proxies.getOrElse(ws0, ws0.asInstanceOf[Operator[Record] with ScalaOperator[_, _]])
-        val newProxies = proxies + (contract.getSolutionSetDelta.asInstanceOf[Operator[Record]] -> s0contract) + (contract.getNextWorkset.asInstanceOf[Operator[Record]] -> ws0contract)
-
-        val freePos1 = globalizeContract(s0, Seq(contract.key.inputFields), proxies, fixedOutputs, freePos)
-        val freePos2 = globalizeContract(ws0, Seq(), proxies, None, freePos1)
-        val freePos3 = globalizeContract(deltaS, Seq(), newProxies, Some(s0contract.getUDF.outputFields), freePos2)
-        val freePos4 = globalizeContract(newWS, Seq(), newProxies, Some(ws0contract.getUDF.outputFields), freePos3)
-
-        contract.getUDF.assignOutputGlobalIndexes(s0contract.getUDF.outputFields)
-
-        freePos4
-      }
-
-      case contract : CoGroupOperator with TwoInputKeyedScalaOperator[_, _, _] => {
-
-        val freePos1 = globalizeContract(contract.getFirstInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.leftInputFields, contract.leftKey.inputFields), proxies, None, freePos)
-        val freePos2 = globalizeContract(contract.getSecondInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.rightInputFields, contract.rightKey.inputFields), proxies, None, freePos1)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos2, fixedOutputs)
-      }
-
-      case contract: CrossOperator with TwoInputScalaOperator[_, _, _] => {
-
-        val freePos1 = globalizeContract(contract.getFirstInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.leftInputFields), proxies, None, freePos)
-        val freePos2 = globalizeContract(contract.getSecondInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.rightInputFields), proxies, None, freePos1)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos2, fixedOutputs)
-      }
-
-      case contract : JoinOperator with TwoInputKeyedScalaOperator[_, _, _] => {
-
-        val freePos1 = globalizeContract(contract.getFirstInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.leftInputFields, contract.leftKey.inputFields), proxies, None, freePos)
-        val freePos2 = globalizeContract(contract.getSecondInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.rightInputFields, contract.rightKey.inputFields), proxies, None, freePos1)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos2, fixedOutputs)
-      }
-
-      case contract : MapOperatorBase[_, _, _] with OneInputScalaOperator[_, _] => {
-
-        val freePos1 = globalizeContract(contract.getInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.inputFields), proxies, None, freePos)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos1, fixedOutputs)
-      }
-
-      case contract : ReduceOperator with OneInputKeyedScalaOperator[_, _] => {
-
-        val freePos1 = globalizeContract(contract.getInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.inputFields, contract.key.inputFields), proxies, None, freePos)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos1, fixedOutputs)
-      }
-
-      // for key-less (global) reducers
-      case contract : ReduceOperator with OneInputScalaOperator[_, _] => {
-
-        val freePos1 = globalizeContract(contract.getInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.inputFields), proxies, None, freePos)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos1, fixedOutputs)
-      }
-
-      case contract : Union[_] with UnionScalaOperator[_] => {
-        
-        val freePos1 = globalizeContract(contract.getFirstInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.leftInputFields), proxies, fixedOutputs, freePos)
-        val freePos2 = globalizeContract(contract.getSecondInput().asInstanceOf[Operator[Record]], Seq(contract.getUDF.rightInputFields), proxies, fixedOutputs, freePos1)
-
-        contract.getUDF.setOutputGlobalIndexes(freePos2, fixedOutputs)
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaPrinter.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaPrinter.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaPrinter.scala
deleted file mode 100644
index 2d758ce..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/GlobalSchemaPrinter.scala
+++ /dev/null
@@ -1,217 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-import org.apache.commons.logging.{LogFactory, Log}
-
-import scala.collection.JavaConversions.collectionAsScalaIterable
-import scala.Array.canBuildFrom
-
-import org.apache.flink.api.common.Plan
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.common.operators.DualInputOperator
-import org.apache.flink.api.common.operators.SingleInputOperator
-import org.apache.flink.api.common.operators.base.{BulkIterationBase => BulkIteration, DeltaIterationBase => DeltaIteration}
-import org.apache.flink.api.java.record.operators.GenericDataSink
-import org.apache.flink.api.common.operators.base.{BulkIterationBase => BulkIteration}
-import org.apache.flink.api.common.operators.base.{DeltaIterationBase => DeltaIteration}
-
-import Extractors.DataSourceNode
-import Extractors.DataSinkNode
-import Extractors.DeltaIterationNode
-import Extractors.JoinNode
-import Extractors.MapNode
-import Extractors.ReduceNode
-import Extractors.UnionNode
-
-object GlobalSchemaPrinter {
-
-  import Extractors._
-
-  private final val LOG: Log = LogFactory.getLog(classOf[GlobalSchemaGenerator])
-
-  def printSchema(plan: Plan): Unit = {
-
-    LOG.debug("### " + plan.getJobName + " ###")
-    plan.getDataSinks.foldLeft(Set[Operator[_]]())(printSchema)
-    LOG.debug("####" + ("#" * plan.getJobName.length) + "####")
-  }
-
-  private def printSchema(visited: Set[Operator[_]], node: Operator[_]): Set[Operator[_]] = {
-
-    visited.contains(node) match {
-
-      case true => visited
-
-      case false => {
-
-        val children = node match {
-          case bi: BulkIteration[_] => List(bi.getInput()) :+ bi.getNextPartialSolution()
-          case wi: DeltaIteration[_, _] => List(wi.getInitialSolutionSet()) :+ wi.getInitialWorkset() :+ wi.getSolutionSetDelta() :+ wi.getNextWorkset()
-          case si : SingleInputOperator[_, _, _] => List(si.getInput())
-          case di : DualInputOperator[_, _, _, _] => List(di.getFirstInput()) :+ di.getSecondInput()
-          case gds : GenericDataSink => List(gds.getInput())
-          case _ => List()
-        }
-        val newVisited = children.foldLeft(visited + node)(printSchema)
-
-        node match {
-          
-          case _ : BulkIteration.PartialSolutionPlaceHolder[_] =>
-          case _ : DeltaIteration.SolutionSetPlaceHolder[_] =>
-          case _ : DeltaIteration.WorksetPlaceHolder[_] =>
-
-          case DataSinkNode(udf, input) => {
-            printInfo(node, "Sink",
-              Seq(),
-              Seq(("", udf.inputFields)),
-              Seq(("", udf.getForwardIndexArrayFrom)),
-              Seq(("", udf.getDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case DataSourceNode(udf) => {
-            printInfo(node, "Source",
-              Seq(),
-              Seq(),
-              Seq(),
-              Seq(),
-              udf.outputFields
-            )
-          }
-
-          case CoGroupNode(udf, leftKey, rightKey, leftInput, rightInput) => {
-            printInfo(node, "CoGroup",
-              Seq(("L", leftKey), ("R", rightKey)),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case CrossNode(udf, leftInput, rightInput) => {
-            printInfo(node, "Cross",
-              Seq(),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case JoinNode(udf, leftKey, rightKey, leftInput, rightInput) => {
-            printInfo(node, "Join",
-              Seq(("L", leftKey), ("R", rightKey)),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case MapNode(udf, input) => {
-            printInfo(node, "Map",
-              Seq(),
-              Seq(("", udf.inputFields)),
-              Seq(("", udf.getForwardIndexArrayFrom)),
-              Seq(("", udf.getDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-          
-          case UnionNode(udf, leftInput, rightInput) => {
-            printInfo(node, "Union",
-              Seq(),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case ReduceNode(udf, key, input) => {
-
-//            val contract = node.asInstanceOf[Reduce4sContract[_, _, _]] 
-//            contract.userCombineCode map { _ =>
-//              printInfo(node, "Combine",
-//                Seq(("", key)),
-//                Seq(("", udf.inputFields)),
-//                Seq(("", contract.combineForwardSet.toArray)),
-//                Seq(("", contract.combineDiscardSet.toArray)),
-//                udf.inputFields
-//              )
-//            }
-
-            printInfo(node, "Reduce",
-              Seq(("", key)),
-              Seq(("", udf.inputFields)),
-              Seq(("", udf.getForwardIndexArrayFrom)),
-              Seq(("", udf.getDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-          case DeltaIterationNode(udf, key, input1, input2) => {
-
-            printInfo(node, "WorksetIterate",
-              Seq(("", key)),
-              Seq(),
-              Seq(),
-              Seq(),
-              udf.outputFields)
-          }
-
-          case BulkIterationNode(udf, input1) => {
-
-            printInfo(node, "BulkIterate",
-              Seq(),
-              Seq(),
-              Seq(),
-              Seq(),
-              udf.outputFields)
-          }
-        }
-
-        newVisited
-      }
-    }
-  }
-
-  private def printInfo(node: Operator[_], kind: String, keys: Seq[(String, FieldSelector)], reads: Seq[(String, FieldSet[_])], forwards: Seq[(String, Array[Int])], discards: Seq[(String, Array[Int])], writes: FieldSet[_]): Unit = {
-
-    def indexesToStrings(pre: String, indexes: Array[Int]) = indexes map {
-      case -1 => "_"
-      case i  => pre + i
-    }
-
-    val formatString = "%s (%s): K{%s}: R[%s] => F[%s] - D[%s] + W[%s]"
-
-    val name = node.getName
-
-    val sKeys = keys flatMap { case (pre, value) => value.selectedFields.toSerializerIndexArray.map(pre + _) } mkString ", "
-    val sReads = reads flatMap { case (pre, value) => indexesToStrings(pre, value.toSerializerIndexArray) } mkString ", "
-    val sForwards = forwards flatMap { case (pre, value) => value.sorted.map(pre + _) } mkString ", "
-    val sDiscards = discards flatMap { case (pre, value) => value.sorted.map(pre + _) } mkString ", "
-    val sWrites = indexesToStrings("", writes.toSerializerIndexArray) mkString ", "
-
-    LOG.debug(formatString.format(name, kind, sKeys, sReads, sForwards, sDiscards, sWrites))
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedFunction.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedFunction.scala
deleted file mode 100644
index 2e9c203..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedFunction.scala
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-import scala.collection.mutable
-import scala.util.Either.MergeableEither
-
-
-abstract class UDF[R] extends Serializable {
-
-  val outputUDT: UDT[R]
-  val outputFields = FieldSet.newOutputSet(outputUDT)
-
-  def getOutputSerializer = outputUDT.getSerializer(outputFields.toSerializerIndexArray)
-
-  def getOutputLength = {
-    val indexes = outputFields.toIndexSet
-    if (indexes.isEmpty) {
-      0 
-    } else {
-      indexes.max + 1
-    }
-  }
-
-  def allocateOutputGlobalIndexes(startPos: Int): Int = {
-
-    outputFields.setGlobalized()
-
-    outputFields.map(_.globalPos).foldLeft(startPos) {
-      case (i, gPos @ GlobalPos.Unknown()) => gPos.setIndex(i); i + 1
-      case (i, _)                          => i
-    }
-    startPos
-  }
-
-  def assignOutputGlobalIndexes(sameAs: FieldSet[Field]): Unit = {
-
-    outputFields.setGlobalized()
-
-    outputFields.foreach {
-      case OutputField(localPos, globalPos) => globalPos.setReference(sameAs(localPos).globalPos)
-    }
-  }
-
-  def setOutputGlobalIndexes(startPos: Int, sameAs: Option[FieldSet[Field]]): Int = sameAs match {
-    case None         => allocateOutputGlobalIndexes(startPos)
-    case Some(sameAs) => assignOutputGlobalIndexes(sameAs); startPos
-  }
-
-  def attachOutputsToInputs(inputFields: FieldSet[InputField]): Unit = {
-
-    inputFields.setGlobalized()
-
-    inputFields.foreach {
-      case InputField(localPos, globalPos) => globalPos.setReference(outputFields(localPos).globalPos)
-    }
-  }
-
-  protected def markFieldCopied(inputGlobalPos: GlobalPos, outputLocalPos: Int): Unit = {
-    val outputField = outputFields(outputLocalPos)
-    outputField.globalPos.setReference(inputGlobalPos)
-    outputField.isUsed = false
-  }
-}
-
-class UDF0[R](val outputUDT: UDT[R]) extends UDF[R]
-
-class UDF1[T, R](val inputUDT: UDT[T], val outputUDT: UDT[R]) extends UDF[R] {
-
-  val inputFields = FieldSet.newInputSet(inputUDT)
-  val forwardSet = mutable.Set[(InputField, OutputField)]()
-  val discardSet = mutable.Set[GlobalPos]()
-
-  def getInputDeserializer = inputUDT.getSerializer(inputFields.toSerializerIndexArray)
-  def getForwardIndexSetFrom = forwardSet.map(_._1.localPos)
-  def getForwardIndexSetTo = forwardSet.map(_._2.localPos)
-  def getForwardIndexArrayFrom = getForwardIndexSetFrom.toArray
-  def getForwardIndexArrayTo = getForwardIndexSetTo.toArray
-  def getDiscardIndexArray = discardSet.map(_.getValue).toArray
-
-  override def getOutputLength = {
-    val forwardMax = if (forwardSet.isEmpty) -1 else forwardSet.map(_._2.localPos).max
-    math.max(super.getOutputLength, forwardMax + 1)
-  }
-
-  def markInputFieldUnread(localPos: Int): Unit = {
-    inputFields(localPos).isUsed = false
-  }
-
-  def markFieldCopied(inputLocalPos: Int, outputLocalPos: Int): Unit = {
-    val inputField = inputFields(inputLocalPos)
-    val inputGlobalPos = inputField.globalPos
-    forwardSet.add((inputField, outputFields(outputLocalPos)))
-    markFieldCopied(inputGlobalPos, outputLocalPos)
-  }
-}
-
-class UDF2[T1, T2, R](val leftInputUDT: UDT[T1], val rightInputUDT: UDT[T2], val outputUDT: UDT[R]) extends UDF[R] {
-
-  val leftInputFields = FieldSet.newInputSet(leftInputUDT)
-  val leftForwardSet = mutable.Set[(InputField, OutputField)]()
-  val leftDiscardSet = mutable.Set[GlobalPos]()
-
-  val rightInputFields = FieldSet.newInputSet(rightInputUDT)
-  val rightForwardSet = mutable.Set[(InputField, OutputField)]()
-  val rightDiscardSet = mutable.Set[GlobalPos]()
-
-  def getLeftInputDeserializer = leftInputUDT.getSerializer(leftInputFields.toSerializerIndexArray)
-  def getLeftForwardIndexSetFrom = leftForwardSet.map(_._1.localPos)
-  def getLeftForwardIndexSetTo = leftForwardSet.map(_._2.localPos)
-  def getLeftForwardIndexArrayFrom = getLeftForwardIndexSetFrom.toArray
-  def getLeftForwardIndexArrayTo = getLeftForwardIndexSetTo.toArray
-  def getLeftDiscardIndexArray = leftDiscardSet.map(_.getValue).toArray
-
-  def getRightInputDeserializer = rightInputUDT.getSerializer(rightInputFields.toSerializerIndexArray)
-  def getRightForwardIndexSetFrom = rightForwardSet.map(_._1.localPos)
-  def getRightForwardIndexSetTo = rightForwardSet.map(_._2.localPos)
-  def getRightForwardIndexArrayFrom = getRightForwardIndexSetFrom.toArray
-  def getRightForwardIndexArrayTo = getRightForwardIndexSetTo.toArray
-  def getRightDiscardIndexArray = rightDiscardSet.map(_.getValue).toArray
-
-  override def getOutputLength = {
-    val leftForwardMax = if (leftForwardSet.isEmpty) -1 else leftForwardSet.map(_._2.localPos).max
-    val rightForwardMax = if (rightForwardSet.isEmpty) -1 else rightForwardSet.map(_._2.localPos).max
-    math.max(super.getOutputLength, math.max(leftForwardMax, rightForwardMax) + 1)
-  }
-
-  private def getInputField(localPos: Either[Int, Int]): InputField = localPos match {
-    case Left(pos)  => leftInputFields(pos)
-    case Right(pos) => rightInputFields(pos)
-  }
-
-  def markInputFieldUnread(localPos: Either[Int, Int]): Unit = {
-    localPos.fold(leftInputFields(_), rightInputFields(_)).isUsed = false
-  }
-
-  def markFieldCopied(inputLocalPos: Either[Int, Int], outputLocalPos: Int): Unit = {
-    val (inputFields, forwardSet) = inputLocalPos.fold(_ => (leftInputFields, leftForwardSet), _ => (rightInputFields, rightForwardSet))
-    val inputField = inputFields(inputLocalPos.merge)
-    val inputGlobalPos = inputField.globalPos
-    forwardSet.add((inputField, outputFields(outputLocalPos)))
-    markFieldCopied(inputGlobalPos, outputLocalPos)
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedType.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedType.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedType.scala
deleted file mode 100644
index aaa3b7b..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/UserDefinedType.scala
+++ /dev/null
@@ -1,133 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis
-
-import scala.language.experimental.macros
-import scala.language.postfixOps
-
-import org.apache.flink.api.scala.codegen.Util
-
-import org.apache.flink.types.{Key => PactKey}
-import org.apache.flink.types.Record
-import org.apache.flink.types.{Value => PactValue}
-import org.apache.flink.types.StringValue
-
-
-abstract class UDT[T] extends Serializable {
-  protected def createSerializer(indexMap: Array[Int]): UDTSerializer[T]
-  val fieldTypes: Array[Class[_ <: org.apache.flink.types.Value]]
-  val udtIdMap: Map[Int, Int]
-  
-  def numFields = fieldTypes.length
-
-  def getSelectionIndices(selection: List[Int]) = { 
-    selection map { udtIdMap.getOrElse(_, -1) }
-  }
-
-  def getKeySet(fields: Seq[Int]): Array[Class[_ <: PactKey[_]]] = {
-    fields map { fieldNum => fieldTypes(fieldNum).asInstanceOf[Class[_ <: PactKey[_]]] } toArray
-  }
-
-  def getSerializer(indexMap: Array[Int]): UDTSerializer[T] = {
-    val ser = createSerializer(indexMap)
-    ser
-  }
-
-  @transient private var defaultSerializer: UDTSerializer[T] = null
-
-  def getSerializerWithDefaultLayout: UDTSerializer[T] = {
-    // This method will be reentrant if T is a recursive type
-    if (defaultSerializer == null) {
-      defaultSerializer = createSerializer((0 until numFields) toArray)
-    }
-    defaultSerializer
-  }
-}
-
-abstract class UDTSerializer[T](val indexMap: Array[Int]) {
-  def serialize(item: T, record: Record)
-  def deserializeRecyclingOff(record: Record): T
-  def deserializeRecyclingOn(record: Record): T
-}
-
-trait UDTLowPriorityImplicits {
-  implicit def createUDT[T]: UDT[T] = macro Util.createUDTImpl[T]
-}
-
-object UDT extends UDTLowPriorityImplicits {
-
-  // UDTs needed by library code
-
-  object NothingUDT extends UDT[Nothing] {
-    override val fieldTypes = Array[Class[_ <: PactValue]]()
-    override val udtIdMap: Map[Int, Int] = Map()
-    override def createSerializer(indexMap: Array[Int]) = throw new UnsupportedOperationException("Cannot create UDTSerializer for type Nothing")
-  }
-
-  object StringUDT extends UDT[String] {
-
-    override val fieldTypes = Array[Class[_ <: PactValue]](classOf[StringValue])
-    override val udtIdMap: Map[Int, Int] = Map()
-
-    override def createSerializer(indexMap: Array[Int]) = new UDTSerializer[String](indexMap) {
-
-      private val index = indexMap(0)
-
-      @transient private var pactField = new StringValue()
-
-//      override def getFieldIndex(selection: Seq[String]): List[Int] = selection match {
-//        case Seq() => List(index)
-////        case _     => invalidSelection(selection)
-//        case _     => throw new RuntimeException("Invalid selection: " + selection)
-//      }
-
-      override def serialize(item: String, record: Record) = {
-        if (index >= 0) {
-          pactField.setValue(item)
-          record.setField(index, pactField)
-        }
-      }
-
-      override def deserializeRecyclingOff(record: Record): String = {
-        if (index >= 0) {
-          record.getFieldInto(index, pactField)
-          pactField.getValue()
-        } else {
-          null
-        }
-      }
-
-      override def deserializeRecyclingOn(record: Record): String = {
-        if (index >= 0) {
-          record.getFieldInto(index, pactField)
-          pactField.getValue()
-        } else {
-          null
-        }
-      }
-
-      private def readObject(in: java.io.ObjectInputStream) = {
-        in.defaultReadObject()
-        pactField = new StringValue()
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/AmbientFieldDetector.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/AmbientFieldDetector.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/AmbientFieldDetector.scala
deleted file mode 100644
index df7d77f..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/AmbientFieldDetector.scala
+++ /dev/null
@@ -1,116 +0,0 @@
-package org.apache.flink.api.scala.analysis.postPass;
-// Comment out because this is not working right now
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-//package org.apache.flink.api.scala.analysis.postPass
-//
-//import scala.collection.mutable
-//import scala.collection.JavaConversions._
-//
-//import org.apache.flink.api.scala.analysis._
-//import org.apache.flink.api.scala.contracts._
-//
-//import org.apache.flink.pact.compiler.plan._
-//import org.apache.flink.pact.compiler.plan.candidate.OptimizedPlan
-//
-//object AmbientFieldDetector {
-//
-//  import Extractors._
-//  import EdgeDependencySets.EdgeDependencySet
-//
-//  def updateAmbientFields(plan: OptimizedPlan, edgeDependencies: Map[PactConnection, EdgeDependencySet], outputPositions: Map[Int, GlobalPos]): Unit = {
-//    plan.getDataSinks.map(_.getSinkNode).foldLeft(Set[OptimizerNode]())(updateAmbientFields(outputPositions, edgeDependencies))
-//  }
-//
-//  private def updateAmbientFields(outputPositions: Map[Int, GlobalPos], edgeDependencies: Map[PactConnection, EdgeDependencySet])(visited: Set[OptimizerNode], node: OptimizerNode): Set[OptimizerNode] = {
-//
-//    visited.contains(node) match {
-//
-//      case true => visited
-//
-//      case false => {
-//        node match {
-//
-//          case _: SinkJoiner | _: BinaryUnionNode =>
-//          case DataSinkNode(udf, input)                       =>
-//          case DataSourceNode(udf)                            =>
-//
-//          case CoGroupNode(udf, _, _, leftInput, rightInput) => {
-//
-//            val leftProvides = edgeDependencies(leftInput).childProvides
-//            val rightProvides = edgeDependencies(rightInput).childProvides
-//            val parentNeeds = edgeDependencies(node.getOutgoingConnections.head).childProvides
-//            val writes = udf.outputFields.toIndexSet
-//
-//            populateSets(udf.leftForwardSet, udf.leftDiscardSet, leftProvides, parentNeeds, writes, outputPositions)
-//            populateSets(udf.rightForwardSet, udf.rightDiscardSet, rightProvides, parentNeeds, writes, outputPositions)
-//          }
-//
-//          case CrossNode(udf, leftInput, rightInput) => {
-//
-//            val leftProvides = edgeDependencies(leftInput).childProvides
-//            val rightProvides = edgeDependencies(rightInput).childProvides
-//            val parentNeeds = edgeDependencies(node.getOutgoingConnections.head).childProvides
-//            val writes = udf.outputFields.toIndexSet
-//
-//            populateSets(udf.leftForwardSet, udf.leftDiscardSet, leftProvides, parentNeeds, writes, outputPositions)
-//            populateSets(udf.rightForwardSet, udf.rightDiscardSet, rightProvides, parentNeeds, writes, outputPositions)
-//          }
-//
-//          case JoinNode(udf, _, _, leftInput, rightInput) => {
-//
-//            val leftProvides = edgeDependencies(leftInput).childProvides
-//            val rightProvides = edgeDependencies(rightInput).childProvides
-//            val parentNeeds = edgeDependencies(node.getOutgoingConnections.head).childProvides
-//            val writes = udf.outputFields.toIndexSet
-//
-//            populateSets(udf.leftForwardSet, udf.leftDiscardSet, leftProvides, parentNeeds, writes, outputPositions)
-//            populateSets(udf.rightForwardSet, udf.rightDiscardSet, rightProvides, parentNeeds, writes, outputPositions)
-//          }
-//
-//          case MapNode(udf, input) => {
-//
-//            val inputProvides = edgeDependencies(input).childProvides
-//            val parentNeeds = edgeDependencies(node.getOutgoingConnections.head).childProvides
-//            val writes = udf.outputFields.toIndexSet
-//
-//            populateSets(udf.forwardSet, udf.discardSet, inputProvides, parentNeeds, writes, outputPositions)
-//          }
-//
-//          case ReduceNode(udf, _, input) => {
-//            val inputProvides = edgeDependencies(input).childProvides
-//            val parentNeeds = edgeDependencies(node.getOutgoingConnections.head).childProvides
-//            val writes = udf.outputFields.toIndexSet
-//
-//            populateSets(udf.forwardSet, udf.discardSet, inputProvides, parentNeeds, writes, outputPositions)
-//          }
-//        }
-//
-//        node.getIncomingConnections.map(_.getSource).foldLeft(visited + node)(updateAmbientFields(outputPositions, edgeDependencies))
-//      }
-//    }
-//  }
-//
-//  private def populateSets(forwards: mutable.Set[GlobalPos], discards: mutable.Set[GlobalPos], childProvides: Set[Int], parentNeeds: Set[Int], writes: Set[Int], outputPositions: Map[Int, GlobalPos]): Unit = {
-//    forwards.clear()
-//    forwards.addAll((parentNeeds -- writes).intersect(childProvides).map(outputPositions(_)))
-//
-//    discards.clear()
-//    discards.addAll((childProvides -- parentNeeds -- writes).intersect(childProvides).map(outputPositions(_)))
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/EdgeDependencySets.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/EdgeDependencySets.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/EdgeDependencySets.scala
deleted file mode 100644
index c92cef5..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/EdgeDependencySets.scala
+++ /dev/null
@@ -1,200 +0,0 @@
-package org.apache.flink.api.scala.analysis.postPass;
-// Comment out because this is not working right now
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-//
-//package org.apache.flink.api.scala.analysis.postPass
-//
-//import scala.collection.JavaConversions._
-//
-//import org.apache.flink.api.scala.analysis._
-//import org.apache.flink.api.scala.contracts._
-//
-//import org.apache.flink.pact.compiler.plan._
-//import org.apache.flink.pact.compiler.plan.candidate.OptimizedPlan
-//
-//object EdgeDependencySets {
-//
-//  import Extractors._
-//
-//  case class EdgeDependencySet(parentNeeds: Set[Int], childProvides: Set[Int] = Set())
-//
-//  def computeEdgeDependencySets(plan: OptimizedPlan, outputSets: Map[OptimizerNode, Set[Int]]): Map[PactConnection, EdgeDependencySet] = {
-//
-//    plan.getDataSinks.map(_.getSinkNode).foldLeft(Map[PactConnection, EdgeDependencySet]())(computeEdgeDependencySets(outputSets))
-//  }
-//
-//  private def computeEdgeDependencySets(outputSets: Map[OptimizerNode, Set[Int]])(edgeDependencySets: Map[PactConnection, EdgeDependencySet], node: OptimizerNode): Map[PactConnection, EdgeDependencySet] = {
-//
-//    // breadth-first traversal: parentNeeds will be None if any parent has not yet been visited
-//    val parentNeeds = node.getOutgoingConnections().foldLeft(Option(Set[Int]())) {
-//      case (None, _)           => None
-//      case (Some(acc), parent) => edgeDependencySets.get(parent) map { acc ++ _.parentNeeds }
-//    }
-//
-//    parentNeeds match {
-//      case None              => edgeDependencySets
-//      case Some(parentNeeds) => computeEdgeDependencySets(node, parentNeeds, outputSets, edgeDependencySets)
-//    }
-//  }
-//
-//  private def computeEdgeDependencySets(node: OptimizerNode, parentNeeds: Set[Int], outputSets: Map[OptimizerNode, Set[Int]], edgeDependencySets: Map[PactConnection, EdgeDependencySet]): Map[PactConnection, EdgeDependencySet] = {
-//
-//    def updateEdges(needs: (PactConnection, Set[Int])*): Map[PactConnection, EdgeDependencySet] = {
-//
-//      val updParents = node.getOutgoingConnections().foldLeft(edgeDependencySets) { (edgeDependencySets, parent) =>
-//        val entry = edgeDependencySets(parent)
-//        edgeDependencySets.updated(parent, entry.copy(childProvides = parentNeeds))
-//      }
-//
-//      needs.foldLeft(updParents) {
-//        case (edgeDependencySets, (inConn, needs)) => {
-//          val updInConn = edgeDependencySets.updated(inConn, EdgeDependencySet(needs))
-//          computeEdgeDependencySets(outputSets)(updInConn, inConn.getSource)
-//        }
-//      }
-//    }
-//
-//    for (udf <- node.getUDF) {
-//
-//      // suppress outputs that aren't needed by any parent
-//      val writeFields = udf.outputFields filter { _.isUsed }
-//      val unused = writeFields filterNot { f => parentNeeds.contains(f.globalPos.getValue) }
-//
-//      for (field <- unused) {
-//        field.isUsed = false
-//        if (field.globalPos.isIndex)
-//          field.globalPos.setIndex(Int.MinValue)
-//      }
-//    }
-//
-//    node match {
-//
-//      case DataSinkNode(udf, input) => {
-//        val needs = udf.inputFields.toIndexSet
-//        updateEdges(input -> needs)
-//      }
-//
-//      case DataSourceNode(udf) => {
-//        updateEdges()
-//      }
-//
-//      case CoGroupNode(udf, leftKey, rightKey, leftInput, rightInput) => {
-//
-//        val leftReads = udf.leftInputFields.toIndexSet ++ leftKey.selectedFields.toIndexSet
-//        val rightReads = udf.rightInputFields.toIndexSet ++ rightKey.selectedFields.toIndexSet
-//        val writes = udf.outputFields.toIndexSet
-//
-//        val parentPreNeeds = parentNeeds -- writes
-//
-//        val parentLeftNeeds = parentPreNeeds.intersect(outputSets(leftInput.getSource))
-//        val parentRightNeeds = parentPreNeeds.intersect(outputSets(rightInput.getSource))
-//
-//        val leftForwards = udf.leftForwardSet.map(_.getValue)
-//        val rightForwards = udf.rightForwardSet.map(_.getValue)
-//
-//        val (leftRes, rightRes) = parentLeftNeeds.intersect(parentRightNeeds).partition {
-//          case index if leftForwards(index) && !rightForwards(index) => true
-//          case index if !leftForwards(index) && rightForwards(index) => false
-//          case _                                                     => throw new UnsupportedOperationException("Schema conflict: cannot forward the same field from both sides of a two-input operator.")
-//        }
-//
-//        val leftNeeds = (parentLeftNeeds -- rightRes) ++ leftReads
-//        val rightNeeds = (parentRightNeeds -- leftRes) ++ rightReads
-//
-//        updateEdges(leftInput -> leftNeeds, rightInput -> rightNeeds)
-//      }
-//
-//      case CrossNode(udf, leftInput, rightInput) => {
-//
-//        val leftReads = udf.leftInputFields.toIndexSet
-//        val rightReads = udf.rightInputFields.toIndexSet
-//        val writes = udf.outputFields.toIndexSet
-//
-//        val parentPreNeeds = parentNeeds -- writes
-//
-//        val parentLeftNeeds = parentPreNeeds.intersect(outputSets(leftInput.getSource))
-//        val parentRightNeeds = parentPreNeeds.intersect(outputSets(rightInput.getSource))
-//
-//        val leftForwards = udf.leftForwardSet.map(_.getValue)
-//        val rightForwards = udf.rightForwardSet.map(_.getValue)
-//
-//        val (leftRes, rightRes) = parentLeftNeeds.intersect(parentRightNeeds).partition {
-//          case index if leftForwards(index) && !rightForwards(index) => true
-//          case index if !leftForwards(index) && rightForwards(index) => false
-//          case _                                                     => throw new UnsupportedOperationException("Schema conflict: cannot forward the same field from both sides of a two-input operator.")
-//        }
-//
-//        val leftNeeds = (parentLeftNeeds -- rightRes) ++ leftReads
-//        val rightNeeds = (parentRightNeeds -- leftRes) ++ rightReads
-//
-//        updateEdges(leftInput -> leftNeeds, rightInput -> rightNeeds)
-//      }
-//
-//      case JoinNode(udf, leftKey, rightKey, leftInput, rightInput) => {
-//
-//        val leftReads = udf.leftInputFields.toIndexSet ++ leftKey.selectedFields.toIndexSet
-//        val rightReads = udf.rightInputFields.toIndexSet ++ rightKey.selectedFields.toIndexSet
-//        val writes = udf.outputFields.toIndexSet
-//
-//        val parentPreNeeds = parentNeeds -- writes
-//
-//        val parentLeftNeeds = parentPreNeeds.intersect(outputSets(leftInput.getSource))
-//        val parentRightNeeds = parentPreNeeds.intersect(outputSets(rightInput.getSource))
-//
-//        val leftForwards = udf.leftForwardSet.map(_.getValue)
-//        val rightForwards = udf.rightForwardSet.map(_.getValue)
-//
-//        val (leftRes, rightRes) = parentLeftNeeds.intersect(parentRightNeeds).partition {
-//          case index if leftForwards(index) && !rightForwards(index) => true
-//          case index if !leftForwards(index) && rightForwards(index) => false
-//          case _                                                     => throw new UnsupportedOperationException("Schema conflict: cannot forward the same field from both sides of a two-input operator.")
-//        }
-//
-//        val leftNeeds = (parentLeftNeeds -- rightRes) ++ leftReads
-//        val rightNeeds = (parentRightNeeds -- leftRes) ++ rightReads
-//
-//        updateEdges(leftInput -> leftNeeds, rightInput -> rightNeeds)
-//      }
-//
-//      case MapNode(udf, input) => {
-//
-//        val reads = udf.inputFields.toIndexSet
-//        val writes = udf.outputFields.toIndexSet
-//
-//        val needs = parentNeeds -- writes ++ reads
-//
-//        updateEdges(input -> needs)
-//      }
-//
-//      case ReduceNode(udf, key, input) => {
-//
-//        val reads = udf.inputFields.toIndexSet ++ key.selectedFields.toIndexSet
-//        val writes = udf.outputFields.toIndexSet
-//
-//        val needs = parentNeeds -- writes ++ reads
-//
-//        updateEdges(input -> needs)
-//      }
-//
-//      case _: SinkJoiner | _: BinaryUnionNode => {
-//        updateEdges(node.getIncomingConnections.map(_ -> parentNeeds): _*)
-//      }
-//    }
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/Extractors.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/Extractors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/Extractors.scala
deleted file mode 100644
index 09e32d5..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/Extractors.scala
+++ /dev/null
@@ -1,167 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis.postPass
-
-import scala.language.implicitConversions
-
-import scala.Some
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis.FieldSelector
-import org.apache.flink.api.scala.analysis.UDF
-import org.apache.flink.api.scala.analysis.UDF0
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.analysis.UDF2
-
-import org.apache.flink.api.java.record.operators.CoGroupOperator
-import org.apache.flink.api.java.record.operators.CrossOperator
-import org.apache.flink.api.java.record.operators.MapOperator
-import org.apache.flink.api.java.record.operators.JoinOperator
-import org.apache.flink.api.java.record.operators.ReduceOperator
-import org.apache.flink.compiler.dag.BinaryUnionNode
-import org.apache.flink.compiler.dag.BulkIterationNode
-import org.apache.flink.compiler.dag.CoGroupNode
-import org.apache.flink.compiler.dag.CrossNode
-import org.apache.flink.compiler.dag.DataSinkNode
-import org.apache.flink.compiler.dag.DataSourceNode
-import org.apache.flink.compiler.dag.CollectorMapNode
-import org.apache.flink.compiler.dag.MatchNode
-import org.apache.flink.compiler.dag.OptimizerNode
-import org.apache.flink.compiler.dag.PactConnection
-import org.apache.flink.compiler.dag.GroupReduceNode
-import org.apache.flink.compiler.dag.SinkJoiner
-import org.apache.flink.compiler.dag.WorksetIterationNode
-import org.apache.flink.api.common.operators.Union
-import org.apache.flink.api.common.operators.base.{BulkIterationBase => BulkIteration, DeltaIterationBase => DeltaIteration, GenericDataSinkBase, GenericDataSourceBase}
-
-object Extractors {
-
-  implicit def nodeToGetUDF(node: OptimizerNode) = new {
-    def getUDF: Option[UDF[_]] = node match {
-      case _: SinkJoiner | _: BinaryUnionNode => None
-      case _ => {
-        Some(node.getPactContract.asInstanceOf[ScalaOperator[_, _]].getUDF)
-      }
-    }
-  }
-
-  object DataSinkNode {
-    def unapply(node: OptimizerNode): Option[(UDF1[_, _], PactConnection)] = node match {
-      case node: DataSinkNode => node.getPactContract match {
-        case contract: GenericDataSinkBase[_] with ScalaOutputOperator[_] => {
-          Some((contract.getUDF, node.getInputConnection))
-        }
-        case _  => None
-      }
-      case _ => None
-    }
-  }
-
-  object DataSourceNode {
-    def unapply(node: OptimizerNode): Option[(UDF0[_])] = node match {
-      case node: DataSourceNode => node.getPactContract() match {
-        case contract: GenericDataSourceBase[_, _] with ScalaOperator[_, _] => Some(contract.getUDF.asInstanceOf[UDF0[_]])
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-
-  object CoGroupNode {
-    def unapply(node: OptimizerNode): Option[(UDF2[_, _, _], FieldSelector, FieldSelector, PactConnection, PactConnection)] = node match {
-      case node: CoGroupNode => node.getPactContract() match {
-        case contract: CoGroupOperator with TwoInputKeyedScalaOperator[_, _, _] => Some((contract.getUDF, contract.leftKey, contract.rightKey, node.getFirstIncomingConnection, node.getSecondIncomingConnection))
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-
-  object CrossNode {
-    def unapply(node: OptimizerNode): Option[(UDF2[_, _, _], PactConnection, PactConnection)] = node match {
-      case node: CrossNode => node.getPactContract match {
-        case contract: CrossOperator with TwoInputScalaOperator[_, _, _] => Some((contract.getUDF, node.getFirstIncomingConnection, node.getSecondIncomingConnection))
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-
-  object JoinNode {
-    def unapply(node: OptimizerNode): Option[(UDF2[_, _, _], FieldSelector, FieldSelector, PactConnection, PactConnection)] = node match {
-      case node: MatchNode => node.getPactContract match {
-        case contract: JoinOperator with TwoInputKeyedScalaOperator[_, _, _] => Some((contract.getUDF, contract.leftKey, contract.rightKey, node.getFirstIncomingConnection, node.getSecondIncomingConnection))
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-
-  object MapNode {
-    def unapply(node: OptimizerNode): Option[(UDF1[_, _], PactConnection)] = node match {
-      case node: CollectorMapNode => node.getPactContract match {
-        case contract: MapOperator with OneInputScalaOperator[_, _] => Some((contract.getUDF, node.getIncomingConnection))
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-  
-  object UnionNode {
-    def unapply(node: OptimizerNode): Option[(UDF2[_, _, _], PactConnection, PactConnection)] = node match {
-      case node: BinaryUnionNode => node.getPactContract match {
-        case contract: Union[_] with UnionScalaOperator[_] => Some((contract.getUDF, node.getFirstIncomingConnection(), node.getSecondIncomingConnection()))
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-
-  object ReduceNode {
-    def unapply(node: OptimizerNode): Option[(UDF1[_, _], FieldSelector, PactConnection)] = node match {
-      case node: GroupReduceNode => node.getPactContract match {
-        case contract: ReduceOperator with OneInputKeyedScalaOperator[_, _] => Some((contract.getUDF, contract.key, node.getIncomingConnection))
-        case contract: ReduceOperator with OneInputScalaOperator[_, _] => Some((contract.getUDF, new FieldSelector(contract.getUDF.inputUDT, Nil), node.getIncomingConnection))
-        case _ => None
-      }
-      case _ => None
-    }
-  }
-  object DeltaIterationNode {
-    def unapply(node: OptimizerNode): Option[(UDF0[_], FieldSelector, PactConnection, PactConnection)] = node match {
-      case node: WorksetIterationNode => node.getPactContract match {
-        case contract: DeltaIteration[_, _] with DeltaIterationScalaOperator[_] => Some((contract.getUDF, contract.key, node.getFirstIncomingConnection(), node.getSecondIncomingConnection()))
-        case _                                  => None
-      }
-      case _ => None
-    }
-  }
-  
-  object BulkIterationNode {
-    def unapply(node: OptimizerNode): Option[(UDF0[_], PactConnection)] = node match {
-      case node: BulkIterationNode => node.getPactContract match {
-        case contract: BulkIteration[_] with BulkIterationScalaOperator[_] => Some((contract.getUDF, node.getIncomingConnection()))
-        case _                                  => None
-      }
-      case _ => None
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaCompactor.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaCompactor.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaCompactor.scala
deleted file mode 100644
index e74695f..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaCompactor.scala
+++ /dev/null
@@ -1,170 +0,0 @@
-package org.apache.flink.api.scala.analysis.postPass;
-// Comment out because this is not working right now
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-//
-//package org.apache.flink.api.scala.analysis.postPass
-//
-//import scala.collection.mutable
-//import scala.collection.JavaConversions._
-//
-//import org.apache.flink.api.scala.analysis._
-//import org.apache.flink.api.scala.contracts._
-//
-//import org.apache.flink.pact.compiler.plan._
-//import org.apache.flink.pact.compiler.plan.candidate.OptimizedPlan
-//
-//object GlobalSchemaCompactor {
-//
-//  import Extractors._
-//
-//  def compactSchema(plan: OptimizedPlan): Unit = {
-//
-//    val (_, conflicts) = plan.getDataSinks.map(_.getSinkNode).foldLeft((Set[OptimizerNode](), Map[GlobalPos, Set[GlobalPos]]())) {
-//      case ((visited, conflicts), node) => findConflicts(node, visited, conflicts)
-//    }
-//
-//    // Reset all position indexes before reassigning them
-//    conflicts.keys.foreach { _.setIndex(Int.MinValue) }
-//
-//    plan.getDataSinks.map(_.getSinkNode).foldLeft(Set[OptimizerNode]())(compactSchema(conflicts))
-//  }
-//
-//  /**
-//   * Two fields are in conflict when they exist in the same place (record) at the same time (plan node).
-//   * If two fields are in conflict, then they must be assigned different indexes.
-//   *
-//   * p1 conflictsWith p2 =
-//   *   Exists(n in Nodes):
-//   *     p1 != p2 &&
-//   *     (
-//   *       (p1 in n.forwards && p2 in n.forwards) ||
-//   *       (p1 in n.forwards && p2 in n.outputs) ||
-//   *       (p2 in n.forwards && p1 in n.outputs)
-//   *     )
-//   */
-//  private def findConflicts(node: OptimizerNode, visited: Set[OptimizerNode], conflicts: Map[GlobalPos, Set[GlobalPos]]): (Set[OptimizerNode], Map[GlobalPos, Set[GlobalPos]]) = {
-//
-//    visited.contains(node) match {
-//
-//      case true => (visited, conflicts)
-//
-//      case false => {
-//
-//        val (forwardPos, outputFields) = node.getUDF match {
-//          case None                     => (Set[GlobalPos](), Set[OutputField]())
-//          case Some(udf: UDF0[_])       => (Set[GlobalPos](), udf.outputFields.toSet)
-//          case Some(udf: UDF1[_, _])    => (udf.forwardSet, udf.outputFields.toSet)
-//          case Some(udf: UDF2[_, _, _]) => (udf.leftForwardSet ++ udf.rightForwardSet, udf.outputFields.toSet)
-//          case _        => (Set[GlobalPos](), Set[OutputField]())
-//        }
-//
-//        // resolve GlobalPos references to the instance that holds the actual index
-//        val forwards = forwardPos map { _.resolve }
-//        val outputs = outputFields filter { _.isUsed } map { _.globalPos.resolve }
-//
-//        val newConflictsF = forwards.foldLeft(conflicts) {
-//          case (conflicts, fPos) => {
-//            // add all other forwards and all outputs to this forward's conflict set
-//            val fConflicts = conflicts.getOrElse(fPos, Set()) ++ (forwards filterNot { _ == fPos }) ++ outputs
-//            conflicts.updated(fPos, fConflicts)
-//          }
-//        }
-//
-//        val newConflictsO = outputs.foldLeft(newConflictsF) {
-//          case (conflicts, oPos) => {
-//            // add all forwards to this output's conflict set
-//            val oConflicts = conflicts.getOrElse(oPos, Set()) ++ forwards
-//            conflicts.updated(oPos, oConflicts)
-//          }
-//        }
-//
-//        node.getIncomingConnections.map(_.getSource).foldLeft((visited + node, newConflictsO)) {
-//          case ((visited, conflicts), node) => findConflicts(node, visited, conflicts)
-//        }
-//      }
-//    }
-//  }
-//
-//  /**
-//   * Assign indexes bottom-up, giving lower values to fields with larger conflict sets.
-//   * This ordering should do a decent job of minimizing the number of gaps between fields.
-//   */
-//  private def compactSchema(conflicts: Map[GlobalPos, Set[GlobalPos]])(visited: Set[OptimizerNode], node: OptimizerNode): Set[OptimizerNode] = {
-//
-//    visited.contains(node) match {
-//
-//      case true => visited
-//
-//      case false => {
-//
-//        val newVisited = node.getIncomingConnections.map(_.getSource).foldLeft(visited + node)(compactSchema(conflicts))
-//
-//        val outputFields = node.getUDF match {
-//          case None      => Seq[OutputField]()
-//          case Some(udf) => udf.outputFields filter { _.isUsed }
-//        }
-//
-//        val outputs = outputFields map {
-//          case field => {
-//            val pos = field.globalPos.resolve
-//            (pos, field.localPos, conflicts(pos) map { _.getValue })
-//          }
-//        } sortBy {
-//          case (_, localPos, posConflicts) => (Int.MaxValue - posConflicts.size, localPos)
-//        }
-//
-//        val initUsed = outputs map { _._1.getValue } filter { _ >= 0 } toSet
-//
-//        val used = outputs.filter(_._1.getValue < 0).foldLeft(initUsed) {
-//          case (used, (pos, _, conflicts)) => {
-//            val index = chooseIndexValue(used ++ conflicts)
-//            pos.setIndex(index)
-//            used + index
-//          }
-//        }
-//
-//        node.getUDF match {
-//          case Some(udf: UDF1[_, _])    => updateDiscards(used, udf.discardSet)
-//          case Some(udf: UDF2[_, _, _]) => updateDiscards(used, udf.leftDiscardSet, udf.rightDiscardSet)
-//          case _                        =>
-//        }
-//
-//        newVisited
-//      }
-//    }
-//  }
-//
-//  private def chooseIndexValue(used: Set[Int]): Int = {
-//    var index = 0
-//    while (used(index)) {
-//      index = index + 1
-//    }
-//    index
-//  }
-//
-//  private def updateDiscards(outputs: Set[Int], discardSets: mutable.Set[GlobalPos]*): Unit = {
-//    for (discardSet <- discardSets) {
-//
-//      val overwrites = discardSet filter { pos => outputs.contains(pos.getValue) } toList
-//
-//      for (pos <- overwrites)
-//        discardSet.remove(pos)
-//    }
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaOptimizer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaOptimizer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaOptimizer.scala
deleted file mode 100644
index 607a41d..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaOptimizer.scala
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis.postPass
-
-import scala.collection.JavaConversions.asScalaBuffer
-import scala.collection.JavaConversions.collectionAsScalaIterable
-
-import org.apache.flink.api.scala.ScalaOperator
-
-import org.apache.flink.compiler.dag.OptimizerNode
-import org.apache.flink.compiler.plan.OptimizedPlan
-
-
-trait GlobalSchemaOptimizer {
-
-  import Extractors._
-
-  def optimizeSchema(plan: OptimizedPlan, compactSchema: Boolean): Unit = {
-
-//    val (outputSets, outputPositions) = OutputSets.computeOutputSets(plan)
-//    val edgeSchemas = EdgeDependencySets.computeEdgeDependencySets(plan, outputSets)
-//
-//    AmbientFieldDetector.updateAmbientFields(plan, edgeSchemas, outputPositions)
-//
-//    if (compactSchema) {
-//      GlobalSchemaCompactor.compactSchema(plan)
-//    }
-
-    GlobalSchemaPrinter.printSchema(plan)
-    
-    plan.getDataSinks.map(_.getSinkNode).foldLeft(Set[OptimizerNode]())(persistConfiguration)
-  }
-
-  private def persistConfiguration(visited: Set[OptimizerNode], node: OptimizerNode): Set[OptimizerNode] = {
-
-    visited.contains(node) match {
-
-      case true => visited
-
-      case false => {
-
-        val children = node.getIncomingConnections.map(_.getSource).toSet
-        val newVisited = children.foldLeft(visited + node)(persistConfiguration)
-
-        node.getPactContract match {
-
-          case c: ScalaOperator[_, _] => c.persistConfiguration(Some(node))
-          case _                    =>
-        }
-
-        newVisited
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaPrinter.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaPrinter.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaPrinter.scala
deleted file mode 100644
index 9e79613..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/GlobalSchemaPrinter.scala
+++ /dev/null
@@ -1,214 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis.postPass
-
-import scala.Array.canBuildFrom
-import scala.collection.JavaConversions.asScalaBuffer
-import scala.collection.JavaConversions.collectionAsScalaIterable
-
-import org.apache.commons.logging.{LogFactory, Log}
-
-import Extractors.CoGroupNode
-import Extractors.CrossNode
-import Extractors.DataSinkNode
-import Extractors.DataSourceNode
-import Extractors.JoinNode
-import Extractors.MapNode
-import Extractors.ReduceNode
-
-import org.apache.flink.api.scala.analysis.FieldSet
-import org.apache.flink.api.scala.analysis.FieldSelector
-
-import org.apache.flink.compiler.dag.BinaryUnionNode
-import org.apache.flink.compiler.dag.OptimizerNode
-import org.apache.flink.compiler.dag.SinkJoiner
-import org.apache.flink.compiler.plan.OptimizedPlan
-import org.apache.flink.api.common.Plan
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.common.operators.SingleInputOperator
-import org.apache.flink.api.common.operators.DualInputOperator
-import org.apache.flink.api.java.record.operators.GenericDataSink
-
-object GlobalSchemaPrinter {
-
-  import Extractors._
-
-  private final val LOG: Log = LogFactory.getLog(classOf[GlobalSchemaOptimizer])
-
-  def printSchema(plan: OptimizedPlan): Unit = {
-
-    LOG.debug("### " + plan.getJobName + " ###")
-    plan.getDataSinks.map(_.getSinkNode).foldLeft(Set[OptimizerNode]())(printSchema)
-    LOG.debug("####" + ("#" * plan.getJobName.length) + "####")
-  }
-
-  private def printSchema(visited: Set[OptimizerNode], node: OptimizerNode): Set[OptimizerNode] = {
-
-    visited.contains(node) match {
-
-      case true => visited
-
-      case false => {
-
-        val children = node.getIncomingConnections.map(_.getSource).toSet
-        val newVisited = children.foldLeft(visited + node)(printSchema)
-
-        node match {
-
-          case _: SinkJoiner | _: BinaryUnionNode =>
-
-          case DataSinkNode(udf, input) => {
-            printInfo(node, "Sink",
-              Seq(),
-              Seq(("", udf.inputFields)),
-              Seq(("", udf.getForwardIndexArrayFrom)),
-              Seq(("", udf.getDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case DataSourceNode(udf) => {
-            printInfo(node, "Source",
-              Seq(),
-              Seq(),
-              Seq(),
-              Seq(),
-              udf.outputFields
-            )
-          }
-
-          case CoGroupNode(udf, leftKey, rightKey, leftInput, rightInput) => {
-            printInfo(node, "CoGroup",
-              Seq(("L", leftKey), ("R", rightKey)),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case CrossNode(udf, leftInput, rightInput) => {
-            printInfo(node, "Cross",
-              Seq(),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case JoinNode(udf, leftKey, rightKey, leftInput, rightInput) => {
-            printInfo(node, "Join",
-              Seq(("L", leftKey), ("R", rightKey)),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case MapNode(udf, input) => {
-            printInfo(node, "Map",
-              Seq(),
-              Seq(("", udf.inputFields)),
-              Seq(("", udf.getForwardIndexArrayFrom)),
-              Seq(("", udf.getDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case UnionNode(udf, leftInput, rightInput) => {
-            printInfo(node, "Union",
-              Seq(),
-              Seq(("L", udf.leftInputFields), ("R", udf.rightInputFields)),
-              Seq(("L", udf.getLeftForwardIndexArrayFrom), ("R", udf.getRightForwardIndexArrayFrom)),
-              Seq(("L", udf.getLeftDiscardIndexArray), ("R", udf.getRightDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-
-          case ReduceNode(udf, key, input) => {
-
-//            val contract = node.asInstanceOf[Reduce4sContract[_, _, _]]
-//            contract.userCombineCode map { _ =>
-//              printInfo(node, "Combine",
-//                Seq(("", key)),
-//                Seq(("", udf.inputFields)),
-//                Seq(("", contract.combineForwardSet.toArray)),
-//                Seq(("", contract.combineDiscardSet.toArray)),
-//                udf.inputFields
-//              )
-//            }
-
-            printInfo(node, "Reduce",
-              Seq(("", key)),
-              Seq(("", udf.inputFields)),
-              Seq(("", udf.getForwardIndexArrayFrom)),
-              Seq(("", udf.getDiscardIndexArray)),
-              udf.outputFields
-            )
-          }
-          case DeltaIterationNode(udf, key, input1, input2) => {
-
-            printInfo(node, "WorksetIterate",
-              Seq(("", key)),
-              Seq(),
-              Seq(),
-              Seq(),
-              udf.outputFields)
-          }
-
-          case BulkIterationNode(udf, input1) => {
-
-            printInfo(node, "BulkIterate",
-              Seq(),
-              Seq(),
-              Seq(),
-              Seq(),
-              udf.outputFields)
-          }
-
-        }
-
-        newVisited
-      }
-    }
-  }
-
-  private def printInfo(node: OptimizerNode, kind: String, keys: Seq[(String, FieldSelector)], reads: Seq[(String, FieldSet[_])], forwards: Seq[(String, Array[Int])], discards: Seq[(String, Array[Int])], writes: FieldSet[_]): Unit = {
-
-    def indexesToStrings(pre: String, indexes: Array[Int]) = indexes map {
-      case -1 => "_"
-      case i  => pre + i
-    }
-
-    val formatString = "%s (%s): K{%s}: R[%s] => F[%s] - D[%s] + W[%s]"
-
-    val name = node.getName
-
-    val sKeys = keys flatMap { case (pre, value) => value.selectedFields.toSerializerIndexArray.map(pre + _) } mkString ", "
-    val sReads = reads flatMap { case (pre, value) => indexesToStrings(pre, value.toSerializerIndexArray) } mkString ", "
-    val sForwards = forwards flatMap { case (pre, value) => value.sorted.map(pre + _) } mkString ", "
-    val sDiscards = discards flatMap { case (pre, value) => value.sorted.map(pre + _) } mkString ", "
-    val sWrites = indexesToStrings("", writes.toSerializerIndexArray) mkString ", "
-
-    LOG.debug(formatString.format(name, kind, sKeys, sReads, sForwards, sDiscards, sWrites))
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/OutputSets.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/OutputSets.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/OutputSets.scala
deleted file mode 100644
index 50ece34..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/analysis/postPass/OutputSets.scala
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.analysis.postPass
-
-import scala.language.reflectiveCalls
-import scala.collection.JavaConversions._
-
-import org.apache.flink.api.scala.analysis._
-
-import org.apache.flink.compiler.dag._
-import org.apache.flink.compiler.plan.OptimizedPlan
-
-
-object OutputSets {
-
-  import Extractors._
-
-  def computeOutputSets(plan: OptimizedPlan): (Map[OptimizerNode, Set[Int]], Map[Int, GlobalPos]) = {
-    
-    val root = plan.getDataSinks.map(s => s.getSinkNode: OptimizerNode).reduceLeft((n1, n2) => new SinkJoiner(n1, n2))
-    val outputSets = computeOutputSets(Map[OptimizerNode, Set[GlobalPos]](), root)
-    val outputPositions = outputSets(root).map(pos => (pos.getValue, pos)).toMap
-    
-    (outputSets.mapValues(_.map(_.getValue)), outputPositions)
-  }
-
-  private def computeOutputSets(outputSets: Map[OptimizerNode, Set[GlobalPos]], node: OptimizerNode): Map[OptimizerNode, Set[GlobalPos]] = {
-
-    outputSets.contains(node) match {
-
-      case true => outputSets
-
-      case false => {
-
-        val children = node.getIncomingConnections.map(_.getSource).toSet
-        val newOutputSets = children.foldLeft(outputSets)(computeOutputSets)
-        
-        val childOutputs = children.map(newOutputSets(_)).flatten
-        val nodeOutputs = node.getUDF map { _.outputFields.filter(_.isUsed).map(_.globalPos).toSet } getOrElse Set()
-        
-        newOutputSets.updated(node, childOutputs ++ nodeOutputs)
-      }
-    }
-  }
-}


[18/60] git commit: Move the Iteration "Operators" to the operators package

Posted by al...@apache.org.
Move the Iteration "Operators" to the operators package

I'm doing this in preparation for the Scala API rework, since then I can just copy over
the operators package and use them for Scala.


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

Branch: refs/heads/master
Commit: dee54cb55b16fe7b4e3e2f89aa35df917a1f92d7
Parents: 6bbe2a0
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Aug 26 16:15:05 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../spargel/java/VertexCentricIteration.java    |   2 +-
 .../spargel/java/SpargelTranslationTest.java    |   8 +-
 .../compiler/BranchingPlansCompilerTest.java    |   2 +-
 .../CachedMatchStrategyCompilerTest.java        |   2 +-
 .../compiler/CoGroupSolutionSetFirstTest.java   |   3 +-
 .../flink/compiler/IterationsCompilerTest.java  |   4 +-
 .../flink/compiler/PipelineBreakerTest.java     |   2 +-
 .../WorksetIterationsJavaApiCompilerTest.java   |   2 +-
 .../flink/example/java/clustering/KMeans.java   |   2 +-
 .../example/java/graph/ConnectedComponents.java |   2 +-
 .../flink/example/java/graph/PageRankBasic.java |   2 +-
 .../java/graph/TransitiveClosureNaive.java      |   2 +-
 .../flink/example/java/ml/LinearRegression.java |   2 +-
 .../flink/api/java/BulkIterationResultSet.java  |  59 -----
 .../java/org/apache/flink/api/java/DataSet.java |  18 +-
 .../apache/flink/api/java/DeltaIteration.java   | 245 -------------------
 .../flink/api/java/DeltaIterationResultSet.java |  79 ------
 .../apache/flink/api/java/IterativeDataSet.java | 148 -----------
 .../java/operators/BulkIterationResultSet.java  |  61 +++++
 .../api/java/operators/CoGroupOperator.java     |   2 +-
 .../api/java/operators/DeltaIteration.java      | 245 +++++++++++++++++++
 .../java/operators/DeltaIterationResultSet.java |  80 ++++++
 .../api/java/operators/IterativeDataSet.java    | 149 +++++++++++
 .../flink/api/java/operators/JoinOperator.java  |   2 +-
 .../api/java/operators/OperatorTranslation.java |   4 -
 .../java/typeutils/runtime/TupleComparator.java |   1 -
 .../DeltaIterationTranslationTest.java          |   2 +-
 ...ultipleJoinsWithSolutionSetCompilerTest.java |   2 +-
 .../iterations/PageRankCompilerTest.java        |   2 +-
 .../BulkIterationWithAllReducerITCase.java      |   2 +-
 .../CoGroupConnectedComponentsSecondITCase.java |   2 +-
 .../DependencyConnectedComponentsITCase.java    |   2 +-
 .../iterative/IterationWithUnionITCase.java     |   2 +-
 .../aggregators/AggregatorsITCase.java          |   4 +-
 ...nentsWithParametrizableAggregatorITCase.java |   2 +-
 ...entsWithParametrizableConvergenceITCase.java |   2 +-
 .../flink/test/util/testjar/KMeansForTest.java  |   2 +-
 37 files changed, 574 insertions(+), 578 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
index 8f9149c..777cf9d 100644
--- a/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
+++ b/flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
@@ -28,7 +28,7 @@ import org.apache.commons.lang3.Validate;
 
 import org.apache.flink.api.common.aggregators.Aggregator;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.functions.RichCoGroupFunction;
 import org.apache.flink.api.java.operators.CoGroupOperator;
 import org.apache.flink.api.java.operators.CustomUnaryOperation;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java b/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
index e862e7c..155aa3f 100644
--- a/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
+++ b/flink-addons/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
@@ -27,15 +27,11 @@ import static org.junit.Assert.fail;
 import org.junit.Test;
 import org.apache.flink.api.common.aggregators.LongSumAggregator;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
-import org.apache.flink.api.java.DeltaIterationResultSet;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIterationResultSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.TwoInputUdfOperator;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.spargel.java.MessageIterator;
-import org.apache.flink.spargel.java.MessagingFunction;
-import org.apache.flink.spargel.java.VertexCentricIteration;
-import org.apache.flink.spargel.java.VertexUpdateFunction;
 
 @SuppressWarnings("serial")
 public class SpargelTranslationTest {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
index 10fa34f..fd2f422 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/BranchingPlansCompilerTest.java
@@ -29,7 +29,7 @@ import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.functions.RichJoinFunction;
 import org.apache.flink.api.java.record.operators.BulkIteration;
 import org.apache.flink.api.java.record.operators.CoGroupOperator;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
index 8226dbf..1deead3 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CachedMatchStrategyCompilerTest.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.base.GenericDataSourceBase;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.functions.RichJoinFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.compiler.dag.TempMode;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
index 2a4d6d8..54af344 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CoGroupSolutionSetFirstTest.java
@@ -24,10 +24,9 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.compiler.CompilerException;
 import org.apache.flink.compiler.plan.Channel;
 import org.apache.flink.compiler.plan.DualInputPlanNode;
 import org.apache.flink.compiler.plan.OptimizedPlan;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
index 828a635..562336f 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/IterationsCompilerTest.java
@@ -25,9 +25,9 @@ import org.junit.Test;
 
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.aggregation.Aggregations;
 import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichJoinFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
index 598f989..71d9b6a 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/PipelineBreakerTest.java
@@ -24,7 +24,7 @@ import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.compiler.plan.BulkIterationPlanNode;
 import org.apache.flink.compiler.plan.DualInputPlanNode;
 import org.apache.flink.compiler.plan.OptimizedPlan;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
index 62d5d9b..a7c6152 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/WorksetIterationsJavaApiCompilerTest.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichJoinFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
index 8767aca..3c31af6 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/KMeans.java
@@ -30,7 +30,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.example.java.clustering.util.KMeansData;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 /**
  * This example implements a basic K-Means clustering algorithm.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
index f0ea7dc..93e8baa 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
@@ -32,7 +32,7 @@ import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.util.Collector;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.example.java.graph.util.ConnectedComponentsData;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
index e6a9272..656c424 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/PageRankBasic.java
@@ -32,7 +32,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.util.Collector;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.example.java.graph.util.PageRankData;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
index e9ba406..281439e 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/TransitiveClosureNaive.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 import org.apache.flink.api.common.ProgramDescription;
 import org.apache.flink.api.java.tuple.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
index 0868732..d65a809 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
@@ -30,7 +30,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.example.java.ml.util.LinearRegressionData;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 /**
  * This example implements a basic Linear Regression  to solve the y = theta0 + theta1*x problem using batch gradient descent algorithm.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/BulkIterationResultSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/BulkIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/BulkIterationResultSet.java
deleted file mode 100644
index 1fd0b6a..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/BulkIterationResultSet.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java;
-
-import org.apache.flink.types.TypeInformation;
-
-public class BulkIterationResultSet<T> extends DataSet<T> {
-
-	private final IterativeDataSet<T> iterationHead;
-
-	private final DataSet<T> nextPartialSolution;
-
-	private final DataSet<?> terminationCriterion;
-
-	BulkIterationResultSet(ExecutionEnvironment context,
-						TypeInformation<T> type,
-						IterativeDataSet<T> iterationHead,
-						DataSet<T> nextPartialSolution) {
-		this(context, type, iterationHead, nextPartialSolution, null);
-	}
-
-	BulkIterationResultSet(ExecutionEnvironment context,
-		TypeInformation<T> type, IterativeDataSet<T> iterationHead,
-		DataSet<T> nextPartialSolution, DataSet<?> terminationCriterion)
-	{
-		super(context, type);
-		this.iterationHead = iterationHead;
-		this.nextPartialSolution = nextPartialSolution;
-		this.terminationCriterion = terminationCriterion;
-	}
-
-	public IterativeDataSet<T> getIterationHead() {
-		return iterationHead;
-	}
-
-	public DataSet<T> getNextPartialSolution() {
-		return nextPartialSolution;
-	}
-
-	public DataSet<?> getTerminationCriterion() {
-		return terminationCriterion;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
index f960abd..3dcd780 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
@@ -45,9 +45,11 @@ import org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets;
 import org.apache.flink.api.java.operators.CrossOperator;
 import org.apache.flink.api.java.operators.CustomUnaryOperation;
 import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.operators.DistinctOperator;
 import org.apache.flink.api.java.operators.FilterOperator;
 import org.apache.flink.api.java.operators.FlatMapOperator;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.operators.JoinOperator.JoinHint;
 import org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets;
 import org.apache.flink.api.java.operators.Keys;
@@ -728,7 +730,7 @@ public abstract class DataSet<T> {
 
 	/**
 	 * Initiates an iterative part of the program that executes multiple times and feeds back data sets.
-	 * The iterative part needs to be closed by calling {@link IterativeDataSet#closeWith(DataSet)}. The data set
+	 * The iterative part needs to be closed by calling {@link org.apache.flink.api.java.operators.IterativeDataSet#closeWith(DataSet)}. The data set
 	 * given to the {@code closeWith(DataSet)} method is the data set that will be fed back and used as the input
 	 * to the next iteration. The return value of the {@code closeWith(DataSet)} method is the resulting
 	 * data set after the iteration has terminated.
@@ -748,13 +750,13 @@ public abstract class DataSet<T> {
 	 * </pre>
 	 * <p>
 	 * The iteration has a maximum number of times that it executes. A dynamic termination can be realized by using a
-	 * termination criterion (see {@link IterativeDataSet#closeWith(DataSet, DataSet)}).
+	 * termination criterion (see {@link org.apache.flink.api.java.operators.IterativeDataSet#closeWith(DataSet, DataSet)}).
 	 * 
 	 * @param maxIterations The maximum number of times that the iteration is executed.
 	 * @return An IterativeDataSet that marks the start of the iterative part and needs to be closed by
-	 *         {@link IterativeDataSet#closeWith(DataSet)}.
+	 *         {@link org.apache.flink.api.java.operators.IterativeDataSet#closeWith(DataSet)}.
 	 * 
-	 * @see IterativeDataSet
+	 * @see org.apache.flink.api.java.operators.IterativeDataSet
 	 */
 	public IterativeDataSet<T> iterate(int maxIterations) {
 		return new IterativeDataSet<T>(getExecutionEnvironment(), getType(), this, maxIterations);
@@ -763,13 +765,13 @@ public abstract class DataSet<T> {
 	/**
 	 * Initiates a delta iteration. A delta iteration is similar to a regular iteration (as started by {@link #iterate(int)},
 	 * but maintains state across the individual iteration steps. The Solution set, which represents the current state
-	 * at the beginning of each iteration can be obtained via {@link DeltaIteration#getSolutionSet()} ()}.
+	 * at the beginning of each iteration can be obtained via {@link org.apache.flink.api.java.operators.DeltaIteration#getSolutionSet()} ()}.
 	 * It can be be accessed by joining (or CoGrouping) with it. The DataSet that represents the workset of an iteration
-	 * can be obtained via {@link DeltaIteration#getWorkset()}.
+	 * can be obtained via {@link org.apache.flink.api.java.operators.DeltaIteration#getWorkset()}.
 	 * The solution set is updated by producing a delta for it, which is merged into the solution set at the end of each
 	 * iteration step.
 	 * <p>
-	 * The delta iteration must be closed by calling {@link DeltaIteration#closeWith(DataSet, DataSet)}. The two
+	 * The delta iteration must be closed by calling {@link org.apache.flink.api.java.operators.DeltaIteration#closeWith(DataSet, DataSet)}. The two
 	 * parameters are the delta for the solution set and the new workset (the data set that will be fed back).
 	 * The return value of the {@code closeWith(DataSet, DataSet)} method is the resulting
 	 * data set after the iteration has terminated. Delta iterations terminate when the feed back data set
@@ -804,7 +806,7 @@ public abstract class DataSet<T> {
 	 * 
 	 * @return The DeltaIteration that marks the start of a delta iteration.
 	 * 
-	 * @see DeltaIteration
+	 * @see org.apache.flink.api.java.operators.DeltaIteration
 	 */
 	public <R> DeltaIteration<T, R> iterateDelta(DataSet<R> workset, int maxIterations, int... keyPositions) {
 		Keys.FieldPositionKeys<T> keys = new Keys.FieldPositionKeys<T>(keyPositions, getType(), false);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java
deleted file mode 100644
index 2f2eae0..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/DeltaIteration.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java;
-
-import java.util.Arrays;
-
-import org.apache.commons.lang3.Validate;
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.aggregators.AggregatorRegistry;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.types.TypeInformation;
-
-/**
- * The DeltaIteration represents the start of a delta iteration. It is created from the DataSet that
- * represents the initial solution set via the {@link DataSet#iterateDelta(DataSet, int, int...)} method.
- * 
- * @param <ST> The data type of the solution set.
- * @param <WT> The data type of the workset (the feedback data set).
- *
- * @see DataSet#iterateDelta(DataSet, int, int...)
- * @see DataSet#iterateDelta(DataSet, int, int[])
- */
-public class DeltaIteration<ST, WT> {
-	
-	private final AggregatorRegistry aggregators = new AggregatorRegistry();
-	
-	private final DataSet<ST> initialSolutionSet;
-	private final DataSet<WT> initialWorkset;
-	
-	private final SolutionSetPlaceHolder<ST> solutionSetPlaceholder;
-	private final WorksetPlaceHolder<WT> worksetPlaceholder;
-
-	private final Keys<ST> keys;
-	
-	private final int maxIterations;
-	
-	private String name;
-	
-	private int parallelism = -1;
-	
-
-	DeltaIteration(ExecutionEnvironment context, TypeInformation<ST> type, DataSet<ST> solutionSet, DataSet<WT> workset, Keys<ST> keys, int maxIterations) {
-		initialSolutionSet = solutionSet;
-		initialWorkset = workset;
-		solutionSetPlaceholder = new SolutionSetPlaceHolder<ST>(context, solutionSet.getType(), this);
-		worksetPlaceholder = new WorksetPlaceHolder<WT>(context, workset.getType());
-		this.keys = keys;
-		this.maxIterations = maxIterations;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Closes the delta iteration. This method defines the end of the delta iteration's function.
-	 * 
-	 * @param solutionSetDelta The delta for the solution set. The delta will be merged into the solution set at the end of
-	 *                         each iteration.
-	 * @param newWorkset The new workset (feedback data set) that will be fed back to the next iteration.
-	 * @return The DataSet that represents the result of the iteration, after the computation has terminated.
-	 * 
-	 * @see DataSet#iterateDelta(DataSet, int, int...)
-	 */
-	public DataSet<ST> closeWith(DataSet<ST> solutionSetDelta, DataSet<WT> newWorkset) {
-		return new DeltaIterationResultSet<ST, WT>(initialSolutionSet.getExecutionEnvironment(),
-				initialSolutionSet.getType(), initialWorkset.getType(), this, solutionSetDelta, newWorkset, keys, maxIterations);
-	}
-
-	/**
-	 * Gets the initial solution set. This is the data set on which the delta iteration was started.
-	 * <p>
-	 * Consider the following example:
-	 * <pre>
-	 * {@code
-	 * DataSet<MyType> solutionSetData = ...;
-	 * DataSet<AnotherType> worksetData = ...;
-	 * 
-	 * DeltaIteration<MyType, AnotherType> iteration = solutionSetData.iteratorDelta(worksetData, 10, ...);
-	 * }
-	 * </pre>
-	 * The <tt>solutionSetData</tt> would be the data set returned by {@code iteration.getInitialSolutionSet();}.
-	 * 
-	 * @return The data set that forms the initial solution set.
-	 */
-	public DataSet<ST> getInitialSolutionSet() {
-		return initialSolutionSet;
-	}
-
-	/**
-	 * Gets the initial workset. This is the data set passed to the method that starts the delta
-	 * iteration.
-	 * <p>
-	 * Consider the following example:
-	 * <pre>
-	 * {@code
-	 * DataSet<MyType> solutionSetData = ...;
-	 * DataSet<AnotherType> worksetData = ...;
-	 * 
-	 * DeltaIteration<MyType, AnotherType> iteration = solutionSetData.iteratorDelta(worksetData, 10, ...);
-	 * }
-	 * </pre>
-	 * The <tt>worksetData</tt> would be the data set returned by {@code iteration.getInitialWorkset();}.
-	 * 
-	 * @return The data set that forms the initial workset.
-	 */
-	public DataSet<WT> getInitialWorkset() {
-		return initialWorkset;
-	}
-
-	/**
-	 * Gets the solution set of the delta iteration. The solution set represents the state that is kept across iterations.
-	 * 
-	 * @return The solution set of the delta iteration.
-	 */
-	public SolutionSetPlaceHolder<ST> getSolutionSet() {
-		return solutionSetPlaceholder;
-	}
-
-	/**
-	 * Gets the working set of the delta iteration. The working set is constructed by the previous iteration.
-	 *
-	 * @return The working set of the delta iteration.
-	 */
-	public WorksetPlaceHolder<WT> getWorkset() {
-		return worksetPlaceholder;
-	}
-
-	/**
-	 * Sets the name for the iteration. The name is displayed in logs and messages.
-	 * 
-	 * @param name The name for the iteration.
-	 * @return The iteration object, for function call chaining.
-	 */
-	public DeltaIteration<ST, WT> name(String name) {
-		this.name = name;
-		return this;
-	}
-	
-	/**
-	 * Gets the name from this iteration.
-	 * 
-	 * @return The name of the iteration.
-	 */
-	public String getName() {
-		return name;
-	}
-	
-	/**
-	 * Sets the degree of parallelism for the iteration.
-	 *
-	 * @param parallelism The degree of parallelism.
-	 * @return The iteration object, for function call chaining.
-	 */
-	public DeltaIteration<ST, WT> parallelism(int parallelism) {
-		Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default).");
-		this.parallelism = parallelism;
-		return this;
-	}
-	
-	/**
-	 * Gets the iteration's degree of parallelism.
-	 * 
-	 * @return The iterations parallelism, or -1, if not set.
-	 */
-	public int getParallelism() {
-		return parallelism;
-	}
-	
-	/**
-	 * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the
-	 * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step,
-	 * the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances.
-	 * The value of an aggregator can be accessed in the next iteration.
-	 * <p>
-	 * Aggregators can be accessed inside a function via the
-	 * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method.
-	 * 
-	 * @param name The name under which the aggregator is registered.
-	 * @param aggregator The aggregator class.
-	 * 
-	 * @return The DeltaIteration itself, to allow chaining function calls.
-	 */
-	public DeltaIteration<ST, WT> registerAggregator(String name, Aggregator<?> aggregator) {
-		this.aggregators.registerAggregator(name, aggregator);
-		return this;
-	}
-	
-	/**
-	 * Gets the registry for aggregators for the iteration.
-	 * 
-	 * @return The registry with all aggregators.
-	 */
-	public AggregatorRegistry getAggregators() {
-		return this.aggregators;
-	}
-	
-	/**
-	 * A {@link DataSet} that acts as a placeholder for the solution set during the iteration.
-	 * 
-	 * @param <ST> The type of the elements in the solution set.
-	 */
-	public static class SolutionSetPlaceHolder<ST> extends DataSet<ST>{
-		
-		private final DeltaIteration<ST, ?> deltaIteration;
-		
-		private SolutionSetPlaceHolder(ExecutionEnvironment context, TypeInformation<ST> type, DeltaIteration<ST, ?> deltaIteration) {
-			super(context, type);
-			this.deltaIteration = deltaIteration;
-		}
-		
-		public void checkJoinKeyFields(int[] keyFields) {
-			int[] ssKeys = deltaIteration.keys.computeLogicalKeyPositions();
-			if (!Arrays.equals(ssKeys, keyFields)) {
-				throw new InvalidProgramException("The solution can only be joined/co-grouped with the same keys as the elements are identified with (here: " + Arrays.toString(ssKeys) + ").");
-			}
-		}
-	}
-
-	/**
-	 * A {@link DataSet} that acts as a placeholder for the workset during the iteration.
-	 *
-	 * @param <WT> The data type of the elements in the workset.
-	 */
-	public static class WorksetPlaceHolder<WT> extends DataSet<WT>{
-		private WorksetPlaceHolder(ExecutionEnvironment context, TypeInformation<WT> type) {
-			super(context, type);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/DeltaIterationResultSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DeltaIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DeltaIterationResultSet.java
deleted file mode 100644
index c41158c..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/DeltaIterationResultSet.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java;
-
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.types.TypeInformation;
-
-public class DeltaIterationResultSet<ST, WT> extends DataSet<ST> {
-
-	private DeltaIteration<ST, WT> iterationHead;
-
-	private DataSet<ST> nextSolutionSet;
-	
-	private DataSet<WT> nextWorkset;
-	
-	private Keys<ST> keys;
-	
-	private int maxIterations;
-	
-	private TypeInformation<WT> typeWS;
-
-	DeltaIterationResultSet(ExecutionEnvironment context,
-							TypeInformation<ST> typeSS,
-							TypeInformation<WT> typeWS,
-							DeltaIteration<ST, WT> iterationHead,
-							DataSet<ST> nextSolutionSet,
-							DataSet<WT> nextWorkset,
-							Keys<ST> keys,
-							int maxIterations)
-	{
-		super(context, typeSS);
-		this.iterationHead = iterationHead;
-		this.nextWorkset = nextWorkset;
-		this.nextSolutionSet = nextSolutionSet;
-		this.keys = keys;
-		this.maxIterations = maxIterations;
-		this.typeWS = typeWS;
-	}
-
-	public DeltaIteration<ST, WT> getIterationHead() {
-		return iterationHead;
-	}
-	
-	public DataSet<ST> getNextSolutionSet() {
-		return nextSolutionSet;
-	}
-
-	public DataSet<WT> getNextWorkset() {
-		return nextWorkset;
-	}
-	
-	public int [] getKeyPositions() {
-		return keys.computeLogicalKeyPositions();
-	}
-	
-	public int getMaxIterations() {
-		return maxIterations;
-	}
-	
-	public TypeInformation<WT> getWorksetType() {
-		return typeWS;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java
deleted file mode 100644
index 5dc0d2e..0000000
--- a/flink-java/src/main/java/org/apache/flink/api/java/IterativeDataSet.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.aggregators.AggregatorRegistry;
-import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
-import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.api.java.operators.SingleInputOperator;
-import org.apache.flink.types.TypeInformation;
-import org.apache.flink.types.Value;
-
-/**
- * The IterativeDataSet represents the start of an iteration. It is created from the DataSet that 
- * represents the initial solution set via the {@link DataSet#iterate(int)} method.
- * 
- * @param <T> The data type of set that is the input and feedback of the iteration.
- *
- * @see DataSet#iterate(int)
- */
-public class IterativeDataSet<T> extends SingleInputOperator<T, T, IterativeDataSet<T>> {
-
-	private final AggregatorRegistry aggregators = new AggregatorRegistry();
-	
-	private int maxIterations;
-
-	IterativeDataSet(ExecutionEnvironment context, TypeInformation<T> type, DataSet<T> input, int maxIterations) {
-		super(input, type);
-		this.maxIterations = maxIterations;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Closes the iteration. This method defines the end of the iterative program part.
-	 * 
-	 * @param iterationResult The data set that will be fed back to the next iteration.
-	 * @return The DataSet that represents the result of the iteration, after the computation has terminated.
-	 * 
-	 * @see DataSet#iterate(int)
-	 */
-	public DataSet<T> closeWith(DataSet<T> iterationResult) {
-		return new BulkIterationResultSet<T>(getExecutionEnvironment(), getType(), this, iterationResult);
-	}
-	
-	/**
-	 * Closes the iteration and specifies a termination criterion. This method defines the end of
-	 * the iterative program part.
-	 * <p>
-	 * The termination criterion is a means of dynamically signaling the iteration to halt. It is expressed via a data
-	 * set that will trigger to halt the loop as soon as the data set is empty. A typical way of using the termination
-	 * criterion is to have a filter that filters out all elements that are considered non-converged. As soon as no more
-	 * such elements exist, the iteration finishes.
-	 * 
-	 * @param iterationResult The data set that will be fed back to the next iteration.
-	 * @return The DataSet that represents the result of the iteration, after the computation has terminated.
-	 * 
-	 * @see DataSet#iterate(int)
-	 */
-	public DataSet<T> closeWith(DataSet<T> iterationResult, DataSet<?> terminationCriterion) {
-		return new BulkIterationResultSet<T>(getExecutionEnvironment(), getType(), this, iterationResult, terminationCriterion);
-	}
-
-	/**
-	 * Gets the maximum number of iterations.
-	 * 
-	 * @return The maximum number of iterations.
-	 */
-	public int getMaxIterations() {
-		return maxIterations;
-	}
-	
-	/**
-	 * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the
-	 * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step,
-	 * the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances.
-	 * The value of an aggregator can be accessed in the next iteration.
-	 * <p>
-	 * Aggregators can be accessed inside a function via the
-	 * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method.
-	 * 
-	 * @param name The name under which the aggregator is registered.
-	 * @param aggregator The aggregator class.
-	 * 
-	 * @return The IterativeDataSet itself, to allow chaining function calls.
-	 */
-	public IterativeDataSet<T> registerAggregator(String name, Aggregator<?> aggregator) {
-		this.aggregators.registerAggregator(name, aggregator);
-		return this;
-	}
-	
-	/**
-	 * Registers an {@link Aggregator} for the iteration together with a {@link ConvergenceCriterion}. For a general description
-	 * of aggregators, see {@link #registerAggregator(String, Aggregator)} and {@link Aggregator}.
-	 * At the end of each iteration, the convergence criterion takes the aggregator's global aggregate value and decided whether
-	 * the iteration should terminate. A typical use case is to have an aggregator that sums up the total error of change
-	 * in an iteration step and have to have a convergence criterion that signals termination as soon as the aggregate value
-	 * is below a certain threshold.
-	 * 
-	 * @param name The name under which the aggregator is registered.
-	 * @param aggregator The aggregator class.
-	 * @param convergenceCheck The convergence criterion.
-	 * 
-	 * @return The IterativeDataSet itself, to allow chaining function calls.
-	 */
-	public <X extends Value> IterativeDataSet<T> registerAggregationConvergenceCriterion(
-			String name, Aggregator<X> aggregator, ConvergenceCriterion<X> convergenceCheck)
-	{
-		this.aggregators.registerAggregationConvergenceCriterion(name, aggregator, convergenceCheck);
-		return this;
-	}
-	
-	/**
-	 * Gets the registry for aggregators. On the registry, one can add {@link Aggregator}s and an aggregator-based 
-	 * {@link ConvergenceCriterion}. This method offers an alternative way to registering the aggregators via
-	 * {@link #registerAggregator(String, Aggregator)} and {@link #registerAggregationConvergenceCriterion(String, Aggregator, ConvergenceCriterion)
-)}.
-	 * 
-	 * @return The registry for aggregators.
-	 */
-	public AggregatorRegistry getAggregators() {
-		return aggregators;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	protected org.apache.flink.api.common.operators.SingleInputOperator<T, T, ?> translateToDataFlow(Operator<T> input) {
-		// All the translation magic happens when the iteration end is encountered.
-		throw new RuntimeException("Error while creating the data flow plan for an iteration: The iteration end was not specified correctly.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java
new file mode 100644
index 0000000..8214174
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/BulkIterationResultSet.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.types.TypeInformation;
+
+public class BulkIterationResultSet<T> extends DataSet<T> {
+
+	private final IterativeDataSet<T> iterationHead;
+
+	private final DataSet<T> nextPartialSolution;
+
+	private final DataSet<?> terminationCriterion;
+
+	BulkIterationResultSet(ExecutionEnvironment context,
+						TypeInformation<T> type,
+						IterativeDataSet<T> iterationHead,
+						DataSet<T> nextPartialSolution) {
+		this(context, type, iterationHead, nextPartialSolution, null);
+	}
+
+	BulkIterationResultSet(ExecutionEnvironment context,
+		TypeInformation<T> type, IterativeDataSet<T> iterationHead,
+		DataSet<T> nextPartialSolution, DataSet<?> terminationCriterion)
+	{
+		super(context, type);
+		this.iterationHead = iterationHead;
+		this.nextPartialSolution = nextPartialSolution;
+		this.terminationCriterion = terminationCriterion;
+	}
+
+	public IterativeDataSet<T> getIterationHead() {
+		return iterationHead;
+	}
+
+	public DataSet<T> getNextPartialSolution() {
+		return nextPartialSolution;
+	}
+
+	public DataSet<?> getTerminationCriterion() {
+		return terminationCriterion;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
index 91ed28d..dd0eba1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.CoGroupOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration.SolutionSetPlaceHolder;
+import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.functions.UnsupportedLambdaExpressionException;
 import org.apache.flink.api.java.operators.Keys.FieldPositionKeys;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
new file mode 100644
index 0000000..803759f
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
@@ -0,0 +1,245 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators;
+
+import java.util.Arrays;
+
+import org.apache.commons.lang3.Validate;
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.aggregators.AggregatorRegistry;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.types.TypeInformation;
+
+/**
+ * The DeltaIteration represents the start of a delta iteration. It is created from the DataSet that
+ * represents the initial solution set via the {@link DataSet#iterateDelta(DataSet, int, int...)} method.
+ * 
+ * @param <ST> The data type of the solution set.
+ * @param <WT> The data type of the workset (the feedback data set).
+ *
+ * @see DataSet#iterateDelta(DataSet, int, int...)
+ * @see DataSet#iterateDelta(DataSet, int, int[])
+ */
+public class DeltaIteration<ST, WT> {
+	
+	private final AggregatorRegistry aggregators = new AggregatorRegistry();
+	
+	private final DataSet<ST> initialSolutionSet;
+	private final DataSet<WT> initialWorkset;
+	
+	private final SolutionSetPlaceHolder<ST> solutionSetPlaceholder;
+	private final WorksetPlaceHolder<WT> worksetPlaceholder;
+
+	private final Keys<ST> keys;
+	
+	private final int maxIterations;
+	
+	private String name;
+	
+	private int parallelism = -1;
+	
+	public DeltaIteration(ExecutionEnvironment context, TypeInformation<ST> type, DataSet<ST> solutionSet, DataSet<WT> workset, Keys<ST> keys, int maxIterations) {
+		initialSolutionSet = solutionSet;
+		initialWorkset = workset;
+		solutionSetPlaceholder = new SolutionSetPlaceHolder<ST>(context, solutionSet.getType(), this);
+		worksetPlaceholder = new WorksetPlaceHolder<WT>(context, workset.getType());
+		this.keys = keys;
+		this.maxIterations = maxIterations;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Closes the delta iteration. This method defines the end of the delta iteration's function.
+	 * 
+	 * @param solutionSetDelta The delta for the solution set. The delta will be merged into the solution set at the end of
+	 *                         each iteration.
+	 * @param newWorkset The new workset (feedback data set) that will be fed back to the next iteration.
+	 * @return The DataSet that represents the result of the iteration, after the computation has terminated.
+	 * 
+	 * @see DataSet#iterateDelta(DataSet, int, int...)
+	 */
+	public DataSet<ST> closeWith(DataSet<ST> solutionSetDelta, DataSet<WT> newWorkset) {
+		return new DeltaIterationResultSet<ST, WT>(initialSolutionSet.getExecutionEnvironment(),
+				initialSolutionSet.getType(), initialWorkset.getType(), this, solutionSetDelta, newWorkset, keys, maxIterations);
+	}
+
+	/**
+	 * Gets the initial solution set. This is the data set on which the delta iteration was started.
+	 * <p>
+	 * Consider the following example:
+	 * <pre>
+	 * {@code
+	 * DataSet<MyType> solutionSetData = ...;
+	 * DataSet<AnotherType> worksetData = ...;
+	 * 
+	 * DeltaIteration<MyType, AnotherType> iteration = solutionSetData.iteratorDelta(worksetData, 10, ...);
+	 * }
+	 * </pre>
+	 * The <tt>solutionSetData</tt> would be the data set returned by {@code iteration.getInitialSolutionSet();}.
+	 * 
+	 * @return The data set that forms the initial solution set.
+	 */
+	public DataSet<ST> getInitialSolutionSet() {
+		return initialSolutionSet;
+	}
+
+	/**
+	 * Gets the initial workset. This is the data set passed to the method that starts the delta
+	 * iteration.
+	 * <p>
+	 * Consider the following example:
+	 * <pre>
+	 * {@code
+	 * DataSet<MyType> solutionSetData = ...;
+	 * DataSet<AnotherType> worksetData = ...;
+	 * 
+	 * DeltaIteration<MyType, AnotherType> iteration = solutionSetData.iteratorDelta(worksetData, 10, ...);
+	 * }
+	 * </pre>
+	 * The <tt>worksetData</tt> would be the data set returned by {@code iteration.getInitialWorkset();}.
+	 * 
+	 * @return The data set that forms the initial workset.
+	 */
+	public DataSet<WT> getInitialWorkset() {
+		return initialWorkset;
+	}
+
+	/**
+	 * Gets the solution set of the delta iteration. The solution set represents the state that is kept across iterations.
+	 * 
+	 * @return The solution set of the delta iteration.
+	 */
+	public SolutionSetPlaceHolder<ST> getSolutionSet() {
+		return solutionSetPlaceholder;
+	}
+
+	/**
+	 * Gets the working set of the delta iteration. The working set is constructed by the previous iteration.
+	 *
+	 * @return The working set of the delta iteration.
+	 */
+	public WorksetPlaceHolder<WT> getWorkset() {
+		return worksetPlaceholder;
+	}
+
+	/**
+	 * Sets the name for the iteration. The name is displayed in logs and messages.
+	 * 
+	 * @param name The name for the iteration.
+	 * @return The iteration object, for function call chaining.
+	 */
+	public DeltaIteration<ST, WT> name(String name) {
+		this.name = name;
+		return this;
+	}
+	
+	/**
+	 * Gets the name from this iteration.
+	 * 
+	 * @return The name of the iteration.
+	 */
+	public String getName() {
+		return name;
+	}
+	
+	/**
+	 * Sets the degree of parallelism for the iteration.
+	 *
+	 * @param parallelism The degree of parallelism.
+	 * @return The iteration object, for function call chaining.
+	 */
+	public DeltaIteration<ST, WT> parallelism(int parallelism) {
+		Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default).");
+		this.parallelism = parallelism;
+		return this;
+	}
+	
+	/**
+	 * Gets the iteration's degree of parallelism.
+	 * 
+	 * @return The iterations parallelism, or -1, if not set.
+	 */
+	public int getParallelism() {
+		return parallelism;
+	}
+	
+	/**
+	 * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the
+	 * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step,
+	 * the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances.
+	 * The value of an aggregator can be accessed in the next iteration.
+	 * <p>
+	 * Aggregators can be accessed inside a function via the
+	 * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method.
+	 * 
+	 * @param name The name under which the aggregator is registered.
+	 * @param aggregator The aggregator class.
+	 * 
+	 * @return The DeltaIteration itself, to allow chaining function calls.
+	 */
+	public DeltaIteration<ST, WT> registerAggregator(String name, Aggregator<?> aggregator) {
+		this.aggregators.registerAggregator(name, aggregator);
+		return this;
+	}
+	
+	/**
+	 * Gets the registry for aggregators for the iteration.
+	 * 
+	 * @return The registry with all aggregators.
+	 */
+	public AggregatorRegistry getAggregators() {
+		return this.aggregators;
+	}
+	
+	/**
+	 * A {@link DataSet} that acts as a placeholder for the solution set during the iteration.
+	 * 
+	 * @param <ST> The type of the elements in the solution set.
+	 */
+	public static class SolutionSetPlaceHolder<ST> extends DataSet<ST>{
+		
+		private final DeltaIteration<ST, ?> deltaIteration;
+		
+		private SolutionSetPlaceHolder(ExecutionEnvironment context, TypeInformation<ST> type, DeltaIteration<ST, ?> deltaIteration) {
+			super(context, type);
+			this.deltaIteration = deltaIteration;
+		}
+		
+		public void checkJoinKeyFields(int[] keyFields) {
+			int[] ssKeys = deltaIteration.keys.computeLogicalKeyPositions();
+			if (!Arrays.equals(ssKeys, keyFields)) {
+				throw new InvalidProgramException("The solution can only be joined/co-grouped with the same keys as the elements are identified with (here: " + Arrays.toString(ssKeys) + ").");
+			}
+		}
+	}
+
+	/**
+	 * A {@link DataSet} that acts as a placeholder for the workset during the iteration.
+	 *
+	 * @param <WT> The data type of the elements in the workset.
+	 */
+	public static class WorksetPlaceHolder<WT> extends DataSet<WT>{
+		private WorksetPlaceHolder(ExecutionEnvironment context, TypeInformation<WT> type) {
+			super(context, type);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java
new file mode 100644
index 0000000..6afa6a9
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.types.TypeInformation;
+
+public class DeltaIterationResultSet<ST, WT> extends DataSet<ST> {
+
+	private DeltaIteration<ST, WT> iterationHead;
+
+	private DataSet<ST> nextSolutionSet;
+	
+	private DataSet<WT> nextWorkset;
+	
+	private Keys<ST> keys;
+	
+	private int maxIterations;
+	
+	private TypeInformation<WT> typeWS;
+
+	DeltaIterationResultSet(ExecutionEnvironment context,
+							TypeInformation<ST> typeSS,
+							TypeInformation<WT> typeWS,
+							DeltaIteration<ST, WT> iterationHead,
+							DataSet<ST> nextSolutionSet,
+							DataSet<WT> nextWorkset,
+							Keys<ST> keys,
+							int maxIterations)
+	{
+		super(context, typeSS);
+		this.iterationHead = iterationHead;
+		this.nextWorkset = nextWorkset;
+		this.nextSolutionSet = nextSolutionSet;
+		this.keys = keys;
+		this.maxIterations = maxIterations;
+		this.typeWS = typeWS;
+	}
+
+	public DeltaIteration<ST, WT> getIterationHead() {
+		return iterationHead;
+	}
+	
+	public DataSet<ST> getNextSolutionSet() {
+		return nextSolutionSet;
+	}
+
+	public DataSet<WT> getNextWorkset() {
+		return nextWorkset;
+	}
+	
+	public int [] getKeyPositions() {
+		return keys.computeLogicalKeyPositions();
+	}
+	
+	public int getMaxIterations() {
+		return maxIterations;
+	}
+	
+	public TypeInformation<WT> getWorksetType() {
+		return typeWS;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java
new file mode 100644
index 0000000..7fc99f6
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.aggregators.AggregatorRegistry;
+import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.types.Value;
+
+/**
+ * The IterativeDataSet represents the start of an iteration. It is created from the DataSet that 
+ * represents the initial solution set via the {@link DataSet#iterate(int)} method.
+ * 
+ * @param <T> The data type of set that is the input and feedback of the iteration.
+ *
+ * @see DataSet#iterate(int)
+ */
+public class IterativeDataSet<T> extends SingleInputOperator<T, T, IterativeDataSet<T>> {
+
+	private final AggregatorRegistry aggregators = new AggregatorRegistry();
+	
+	private int maxIterations;
+
+	public IterativeDataSet(ExecutionEnvironment context, TypeInformation<T> type, DataSet<T> input, int maxIterations) {
+		super(input, type);
+		this.maxIterations = maxIterations;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Closes the iteration. This method defines the end of the iterative program part.
+	 * 
+	 * @param iterationResult The data set that will be fed back to the next iteration.
+	 * @return The DataSet that represents the result of the iteration, after the computation has terminated.
+	 * 
+	 * @see DataSet#iterate(int)
+	 */
+	public DataSet<T> closeWith(DataSet<T> iterationResult) {
+		return new BulkIterationResultSet<T>(getExecutionEnvironment(), getType(), this, iterationResult);
+	}
+	
+	/**
+	 * Closes the iteration and specifies a termination criterion. This method defines the end of
+	 * the iterative program part.
+	 * <p>
+	 * The termination criterion is a means of dynamically signaling the iteration to halt. It is expressed via a data
+	 * set that will trigger to halt the loop as soon as the data set is empty. A typical way of using the termination
+	 * criterion is to have a filter that filters out all elements that are considered non-converged. As soon as no more
+	 * such elements exist, the iteration finishes.
+	 * 
+	 * @param iterationResult The data set that will be fed back to the next iteration.
+	 * @return The DataSet that represents the result of the iteration, after the computation has terminated.
+	 * 
+	 * @see DataSet#iterate(int)
+	 */
+	public DataSet<T> closeWith(DataSet<T> iterationResult, DataSet<?> terminationCriterion) {
+		return new BulkIterationResultSet<T>(getExecutionEnvironment(), getType(), this, iterationResult, terminationCriterion);
+	}
+
+	/**
+	 * Gets the maximum number of iterations.
+	 * 
+	 * @return The maximum number of iterations.
+	 */
+	public int getMaxIterations() {
+		return maxIterations;
+	}
+	
+	/**
+	 * Registers an {@link Aggregator} for the iteration. Aggregators can be used to maintain simple statistics during the
+	 * iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step,
+	 * the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances.
+	 * The value of an aggregator can be accessed in the next iteration.
+	 * <p>
+	 * Aggregators can be accessed inside a function via the
+	 * {@link org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext()} method.
+	 * 
+	 * @param name The name under which the aggregator is registered.
+	 * @param aggregator The aggregator class.
+	 * 
+	 * @return The IterativeDataSet itself, to allow chaining function calls.
+	 */
+	public IterativeDataSet<T> registerAggregator(String name, Aggregator<?> aggregator) {
+		this.aggregators.registerAggregator(name, aggregator);
+		return this;
+	}
+	
+	/**
+	 * Registers an {@link Aggregator} for the iteration together with a {@link ConvergenceCriterion}. For a general description
+	 * of aggregators, see {@link #registerAggregator(String, Aggregator)} and {@link Aggregator}.
+	 * At the end of each iteration, the convergence criterion takes the aggregator's global aggregate value and decided whether
+	 * the iteration should terminate. A typical use case is to have an aggregator that sums up the total error of change
+	 * in an iteration step and have to have a convergence criterion that signals termination as soon as the aggregate value
+	 * is below a certain threshold.
+	 * 
+	 * @param name The name under which the aggregator is registered.
+	 * @param aggregator The aggregator class.
+	 * @param convergenceCheck The convergence criterion.
+	 * 
+	 * @return The IterativeDataSet itself, to allow chaining function calls.
+	 */
+	public <X extends Value> IterativeDataSet<T> registerAggregationConvergenceCriterion(
+			String name, Aggregator<X> aggregator, ConvergenceCriterion<X> convergenceCheck)
+	{
+		this.aggregators.registerAggregationConvergenceCriterion(name, aggregator, convergenceCheck);
+		return this;
+	}
+	
+	/**
+	 * Gets the registry for aggregators. On the registry, one can add {@link Aggregator}s and an aggregator-based 
+	 * {@link ConvergenceCriterion}. This method offers an alternative way to registering the aggregators via
+	 * {@link #registerAggregator(String, Aggregator)} and {@link #registerAggregationConvergenceCriterion(String, Aggregator, ConvergenceCriterion)
+)}.
+	 * 
+	 * @return The registry for aggregators.
+	 */
+	public AggregatorRegistry getAggregators() {
+		return aggregators;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	protected org.apache.flink.api.common.operators.SingleInputOperator<T, T, ?> translateToDataFlow(Operator<T> input) {
+		// All the translation magic happens when the iteration end is encountered.
+		throw new RuntimeException("Error while creating the data flow plan for an iteration: The iteration end was not specified correctly.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index 15447d8..6d74474 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -33,7 +33,7 @@ import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.JoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration.SolutionSetPlaceHolder;
+import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder;
 import org.apache.flink.api.java.functions.RichFlatJoinFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.functions.SemanticPropUtil;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index bf7975f..e44302a 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -18,11 +18,7 @@
 
 package org.apache.flink.api.java.operators;
 
-import org.apache.flink.api.java.BulkIterationResultSet;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
-import org.apache.flink.api.java.DeltaIterationResultSet;
-import org.apache.flink.api.java.IterativeDataSet;
 
 import java.util.ArrayList;
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
index 7b3976f..8783ede 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.api.java.typeutils.runtime;
 
-
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
index 9fcf963..71eb682 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
@@ -33,7 +33,7 @@ import org.apache.flink.api.common.operators.base.GenericDataSinkBase;
 import org.apache.flink.api.common.operators.base.JoinOperatorBase;
 import org.apache.flink.api.common.operators.base.MapOperatorBase;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.RichCoGroupFunction;
 import org.apache.flink.api.java.functions.RichJoinFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
index e0a96b3..1d7cfa6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
@@ -34,7 +34,7 @@ import org.apache.flink.test.compiler.util.CompilerTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.compiler.plan.DualInputPlanNode;
 import org.apache.flink.compiler.plan.OptimizedPlan;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
index 65cbe24..241a376 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
@@ -43,7 +43,7 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 public class PageRankCompilerTest extends CompilerTestBase{
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
index b442b33..1d0b765 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
@@ -31,7 +31,7 @@ import org.junit.Assert;
 
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
index c6175b2..e4faa77 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
@@ -31,7 +31,7 @@ import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.example.java.graph.ConnectedComponents.DuplicateValue;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
index a62763a..a0d26db 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
@@ -29,7 +29,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
index 2a130ed..74331a7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.recordJobs.kmeans.udfs.PointInFormat;
 import org.apache.flink.test.recordJobs.kmeans.udfs.PointOutFormat;
 import org.apache.flink.test.util.JavaProgramTestBase;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
index 9ea33d7..12eb87b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
@@ -40,9 +40,9 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 /**
  * Test the functionality of aggregators in bulk and delta iterative cases.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
index 8097f28..a9f01fb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
@@ -33,7 +33,7 @@ import org.apache.flink.util.Collector;
 import org.junit.Assert;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 /**
  * Connected Components test case that uses a parameterizable aggregator

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
index 6678940..5e6be50 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
@@ -33,7 +33,7 @@ import org.apache.flink.types.LongValue;
 import org.apache.flink.util.Collector;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/dee54cb5/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
index 917615d..5390574 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
@@ -33,7 +33,7 @@ import org.apache.flink.configuration.Configuration;
 
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.IterativeDataSet;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 
 @SuppressWarnings("serial")
 public class KMeansForTest implements Program {


[11/60] git commit: Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
Rewrite the Scala API as (somewhat) thin Layer on Java API

Don't bother looking at the diff, this is almost a complete rewrite of
the previous Scala API. This uses all the work put into the Java API,
such as TypeInformation, the serializers and comparators and the
operators.

The Scala DataSet and ExecutionEnvironment wrap their respective Java
equivalents. TypeInformation is generated by a macro that uses
TypeInformationGen and other macro support classes. The Java
TypeExtractor is completely bypassed but the TypeInformation and
sub-classes are created by the Scala type analyzer. There is special
support for Scala Tuples in the form of ScalaTupleTypeInfo,
ScalaTupleSerializer, and ScalaTupleComparator.

This also adds tests to flink-scala that are ports of the tests in
flink-java.

There are not yet any Scala specific tests in flink-tests. All the
scala example ITCases are commented out, as well as the examples
themselves. Those will be uncommented once the examples are ported. This
will happen in separate commits.


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

Branch: refs/heads/master
Commit: b8131fa75361519f5338eef8cd7dec6e31982dc1
Parents: 568dff1
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Sep 4 20:51:43 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../common/typeutils/SerializerTestBase.java    |   2 +-
 .../java/clustering/util/KMeansData.java        | 115 +--
 .../java/wordcount/util/WordCountData.java      |  77 +-
 flink-examples/flink-scala-examples/pom.xml     |  10 +-
 .../examples/scala/clustering/KMeans.scala      | 248 ++++++
 .../scala/datamining/BatchGradientDescent.scala | 166 ++--
 .../examples/scala/datamining/KMeans.scala      | 113 ---
 .../scala/graph/ComputeEdgeDegrees.scala        | 246 +++---
 .../scala/graph/ConnectedComponents.scala       | 180 ++--
 .../graph/EnumTrianglesOnEdgesWithDegrees.scala | 250 +++---
 .../flink/examples/scala/graph/LineRank.scala   | 194 ++---
 .../flink/examples/scala/graph/PageRank.scala   | 216 ++---
 .../scala/graph/PageRankWithWeight.scala        | 216 ++---
 .../scala/graph/TransitiveClosureNaive.scala    | 174 ++--
 .../scala/iterative/TerminationCriterion.scala  | 156 ++--
 .../scala/relational/RelationalQuery.scala      | 214 ++---
 .../scala/relational/WebLogAnalysis.scala       | 300 +++----
 .../examples/scala/testing/KMeansForTest.scala  | 210 ++---
 .../examples/scala/wordcount/WordCount.scala    | 108 ++-
 .../scala/wordcount/WordCountWithCount.scala    | 126 +--
 .../WordCountWithUserDefinedType.scala          | 118 +--
 flink-java/pom.xml                              |  17 +
 .../api/java/operators/AggregateOperator.java   |   8 +-
 .../flink/api/java/operators/CrossOperator.java |   2 +-
 .../flink/api/java/operators/Grouping.java      |   4 +-
 .../flink/api/java/operators/JoinOperator.java  |   8 +-
 .../apache/flink/api/java/operators/Keys.java   |   3 +-
 .../java/typeutils/runtime/TupleSerializer.java |  16 +
 .../typeutils/runtime/TupleSerializerBase.java  |   8 +
 .../main/resources/archetype-resources/pom.xml  |   4 +-
 .../src/main/scala/Job.scala                    | 139 +--
 .../src/main/scala/WordCountJob.scala           |  37 +
 flink-scala/pom.xml                             |  18 +-
 .../flink/api/scala/operators/Annotations.java  | 214 -----
 .../scala/operators/ScalaAggregateOperator.java | 316 +++++++
 .../scala/operators/ScalaCsvInputFormat.java    | 203 +++++
 .../scala/operators/ScalaCsvOutputFormat.java   | 228 +++++
 .../apache/flink/api/scala/AnnotationUtil.scala | 219 -----
 .../apache/flink/api/scala/CompilerHints.scala  | 386 ---------
 .../org/apache/flink/api/scala/DataSet.scala    | 867 +++++++++++++++++--
 .../org/apache/flink/api/scala/DataSink.scala   |  68 --
 .../org/apache/flink/api/scala/DataSource.scala | 124 ---
 .../flink/api/scala/ExecutionEnvironment.scala  | 413 +++++++++
 .../apache/flink/api/scala/GroupedDataSet.scala | 270 ++++++
 .../apache/flink/api/scala/ScalaOperator.scala  | 136 ---
 .../org/apache/flink/api/scala/ScalaPlan.scala  |  95 --
 .../flink/api/scala/analysis/Extractors.scala   | 114 ---
 .../api/scala/analysis/FieldSelector.scala      |  44 -
 .../api/scala/analysis/GlobalSchemaFields.scala | 134 ---
 .../scala/analysis/GlobalSchemaGenerator.scala  | 178 ----
 .../scala/analysis/GlobalSchemaPrinter.scala    | 217 -----
 .../scala/analysis/UserDefinedFunction.scala    | 162 ----
 .../api/scala/analysis/UserDefinedType.scala    | 133 ---
 .../postPass/AmbientFieldDetector.scala         | 116 ---
 .../analysis/postPass/EdgeDependencySets.scala  | 200 -----
 .../scala/analysis/postPass/Extractors.scala    | 167 ----
 .../postPass/GlobalSchemaCompactor.scala        | 170 ----
 .../postPass/GlobalSchemaOptimizer.scala        |  72 --
 .../analysis/postPass/GlobalSchemaPrinter.scala | 214 -----
 .../scala/analysis/postPass/OutputSets.scala    |  62 --
 .../apache/flink/api/scala/coGroupDataSet.scala | 230 +++++
 .../flink/api/scala/codegen/Counter.scala       |   2 +-
 .../scala/codegen/DeserializeMethodGen.scala    | 261 ------
 .../apache/flink/api/scala/codegen/Logger.scala | 118 ---
 .../api/scala/codegen/MacroContextHolder.scala  |  16 +-
 .../api/scala/codegen/SelectionExtractor.scala  | 184 ----
 .../api/scala/codegen/SerializeMethodGen.scala  | 226 -----
 .../flink/api/scala/codegen/SerializerGen.scala | 328 -------
 .../flink/api/scala/codegen/TreeGen.scala       |  80 +-
 .../flink/api/scala/codegen/TypeAnalyzer.scala  | 382 ++++++++
 .../api/scala/codegen/TypeDescriptors.scala     | 186 ++++
 .../api/scala/codegen/TypeInformationGen.scala  | 184 ++++
 .../flink/api/scala/codegen/UDTAnalyzer.scala   | 344 --------
 .../api/scala/codegen/UDTDescriptors.scala      | 158 ----
 .../apache/flink/api/scala/codegen/UDTGen.scala |  92 --
 .../apache/flink/api/scala/codegen/Util.scala   |  49 --
 .../apache/flink/api/scala/crossDataSet.scala   | 132 +++
 .../api/scala/functions/CoGroupFunction.scala   |  92 --
 .../api/scala/functions/CrossFunction.scala     |  66 --
 .../scala/functions/DeserializingIterator.scala |  61 --
 .../api/scala/functions/JoinFunction.scala      |  86 --
 .../flink/api/scala/functions/MapFunction.scala |  83 --
 .../api/scala/functions/ReduceFunction.scala    | 102 ---
 .../apache/flink/api/scala/joinDataSet.scala    | 232 +++++
 .../api/scala/operators/ClosureCleaner.scala    | 227 -----
 .../api/scala/operators/CoGroupOperator.scala   | 242 ------
 .../api/scala/operators/CopyOperator.scala      |  84 --
 .../api/scala/operators/CrossOperator.scala     | 159 ----
 .../api/scala/operators/DataSinkMacros.scala    | 346 --------
 .../api/scala/operators/DataSourceMacros.scala  | 414 ---------
 .../api/scala/operators/IterateOperators.scala  | 235 -----
 .../api/scala/operators/JoinOperator.scala      | 296 -------
 .../flink/api/scala/operators/MapOperator.scala | 218 -----
 .../api/scala/operators/ReduceOperator.scala    | 363 --------
 .../api/scala/operators/UnionOperator.scala     |  44 -
 .../flink/api/scala/operators/package.scala     |  31 -
 .../org/apache/flink/api/scala/package.scala    |  34 +
 .../scala/typeutils/ScalaTupleComparator.scala  | 149 ++++
 .../scala/typeutils/ScalaTupleSerializer.scala  |  66 ++
 .../scala/typeutils/ScalaTupleTypeInfo.scala    |  80 ++
 .../flink/api/scala/typeutils/TypeUtils.scala   |  31 +
 .../api/scala/unfinishedKeyPairOperation.scala  | 115 +++
 .../api/scala/CollectionDataSourceTest.scala    |  53 --
 .../scala/DeltaIterationSanityCheckTest.scala   | 384 ++++----
 .../scala/io/CollectionInputFormatTest.scala    | 133 +++
 .../flink/api/scala/io/CsvInputFormatTest.scala | 288 ++++++
 .../scala/operators/AggregateOperatorTest.scala |  93 ++
 .../scala/operators/CoGroupOperatorTest.scala   | 213 +++++
 .../flink/api/scala/operators/CustomType.scala  |  33 +
 .../scala/operators/DistinctOperatorTest.scala  | 103 +++
 .../api/scala/operators/GroupingTest.scala      | 182 ++++
 .../api/scala/operators/JoinOperatorTest.scala  | 200 +++++
 .../translation/AggregateTranslationTest.scala  |  60 ++
 .../DeltaIterationTranslationTest.scala         | 250 ++++++
 .../translation/DistinctTranslationTest.scala   |  51 ++
 .../translation/ReduceTranslationTest.scala     | 136 +++
 .../runtime/GenericPairComparatorTest.scala     |  76 ++
 .../scala/runtime/TupleComparatorILD2Test.scala |  56 ++
 .../scala/runtime/TupleComparatorILD3Test.scala |  54 ++
 .../runtime/TupleComparatorILDC3Test.scala      |  54 ++
 .../runtime/TupleComparatorILDX1Test.scala      |  54 ++
 .../runtime/TupleComparatorILDXC2Test.scala     |  54 ++
 .../scala/runtime/TupleComparatorISD1Test.scala |  53 ++
 .../scala/runtime/TupleComparatorISD2Test.scala |  53 ++
 .../scala/runtime/TupleComparatorISD3Test.scala |  53 ++
 .../api/scala/runtime/TupleSerializerTest.scala | 190 ++++
 .../runtime/TupleSerializerTestInstance.scala   |  90 ++
 .../tuple/base/PairComparatorTestBase.scala     | 102 +++
 .../tuple/base/TupleComparatorTestBase.scala    |  31 +
 .../scala/types/TypeInformationGenTest.scala    | 258 ++++++
 .../ComputeEdgeDegreesITCase.java               |  78 +-
 .../ConnectedComponentsITCase.java              |  68 +-
 .../EnumTrianglesOnEdgesWithDegreesITCase.java  |  78 +-
 .../IterativeKMeansITCase.java                  |  58 --
 .../RelationalQueryITCase.java                  |  98 +--
 .../TransitiveClosureNaiveITCase.java           | 108 +--
 .../WebLogAnalysisITCase.java                   |  64 +-
 .../exampleScalaPrograms/WordCountITCase.java   |  31 +-
 .../WordCountPactValueITCase.java               |  66 +-
 .../WordCountWithCountFunctionITCase.java       |  62 +-
 .../flink/test/iterative/LineRankITCase.java    | 184 ++--
 141 files changed, 9634 insertions(+), 10671 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
index 0f611ed..a692ffc 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
@@ -341,7 +341,7 @@ public abstract class SerializerTestBase<T> {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	private TypeSerializer<T> getSerializer() {
+	protected TypeSerializer<T> getSerializer() {
 		TypeSerializer<T> serializer = createSerializer();
 		if (serializer == null) {
 			throw new RuntimeException("Test case corrupt. Returns null as serializer.");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
index 2966774..97325dc 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/clustering/util/KMeansData.java
@@ -24,6 +24,9 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.example.java.clustering.KMeans.Centroid;
 import org.apache.flink.example.java.clustering.KMeans.Point;
 
+import java.util.LinkedList;
+import java.util.List;
+
 /**
  * Provides the default data sets used for the K-Means example program.
  * The default data sets are used, if no parameters are given to the program.
@@ -31,60 +34,72 @@ import org.apache.flink.example.java.clustering.KMeans.Point;
  */
 public class KMeansData {
 
+	// We have the data as object arrays so that we can also generate Scala Data Sources from it.
+	public static final Object[][] CENTROIDS = new Object[][] {
+		new Object[] {1, -31.85, -44.77},
+		new Object[]{2, 35.16, 17.46},
+		new Object[]{3, -5.16, 21.93},
+		new Object[]{4, -24.06, 6.81}
+	};
+
+	public static final Object[][] POINTS = new Object[][] {
+		new Object[] {-14.22, -48.01},
+		new Object[] {-22.78, 37.10},
+		new Object[] {56.18, -42.99},
+		new Object[] {35.04, 50.29},
+		new Object[] {-9.53, -46.26},
+		new Object[] {-34.35, 48.25},
+		new Object[] {55.82, -57.49},
+		new Object[] {21.03, 54.64},
+		new Object[] {-13.63, -42.26},
+		new Object[] {-36.57, 32.63},
+		new Object[] {50.65, -52.40},
+		new Object[] {24.48, 34.04},
+		new Object[] {-2.69, -36.02},
+		new Object[] {-38.80, 36.58},
+		new Object[] {24.00, -53.74},
+		new Object[] {32.41, 24.96},
+		new Object[] {-4.32, -56.92},
+		new Object[] {-22.68, 29.42},
+		new Object[] {59.02, -39.56},
+		new Object[] {24.47, 45.07},
+		new Object[] {5.23, -41.20},
+		new Object[] {-23.00, 38.15},
+		new Object[] {44.55, -51.50},
+		new Object[] {14.62, 59.06},
+		new Object[] {7.41, -56.05},
+		new Object[] {-26.63, 28.97},
+		new Object[] {47.37, -44.72},
+		new Object[] {29.07, 51.06},
+		new Object[] {0.59, -31.89},
+		new Object[] {-39.09, 20.78},
+		new Object[] {42.97, -48.98},
+		new Object[] {34.36, 49.08},
+		new Object[] {-21.91, -49.01},
+		new Object[] {-46.68, 46.04},
+		new Object[] {48.52, -43.67},
+		new Object[] {30.05, 49.25},
+		new Object[] {4.03, -43.56},
+		new Object[] {-37.85, 41.72},
+		new Object[] {38.24, -48.32},
+		new Object[] {20.83, 57.85}
+	};
+
 	public static DataSet<Centroid> getDefaultCentroidDataSet(ExecutionEnvironment env) {
-		
-		return env.fromElements(
-				new Centroid(1, -31.85, -44.77),
-				new Centroid(2, 35.16, 17.46),
-				new Centroid(3, -5.16, 21.93),
-				new Centroid(4, -24.06, 6.81)
-				);
+		List<Centroid> centroidList = new LinkedList<Centroid>();
+		for (Object[] centroid : CENTROIDS) {
+			centroidList.add(
+					new Centroid((Integer) centroid[0], (Double) centroid[1], (Double) centroid[2]));
+		}
+		return env.fromCollection(centroidList);
 	}
 	
 	public static DataSet<Point> getDefaultPointDataSet(ExecutionEnvironment env) {
-		
-		return env.fromElements(
-				new Point(-14.22, -48.01),
-				new Point(-22.78, 37.10),
-				new Point(56.18, -42.99),
-				new Point(35.04, 50.29),
-				new Point(-9.53, -46.26),
-				new Point(-34.35, 48.25),
-				new Point(55.82, -57.49),
-				new Point(21.03, 54.64),
-				new Point(-13.63, -42.26),
-				new Point(-36.57, 32.63),
-				new Point(50.65, -52.40),
-				new Point(24.48, 34.04),
-				new Point(-2.69, -36.02),
-				new Point(-38.80, 36.58),
-				new Point(24.00, -53.74),
-				new Point(32.41, 24.96),
-				new Point(-4.32, -56.92),
-				new Point(-22.68, 29.42),
-				new Point(59.02, -39.56),
-				new Point(24.47, 45.07),
-				new Point(5.23, -41.20),
-				new Point(-23.00, 38.15),
-				new Point(44.55, -51.50),
-				new Point(14.62, 59.06),
-				new Point(7.41, -56.05),
-				new Point(-26.63, 28.97),
-				new Point(47.37, -44.72),
-				new Point(29.07, 51.06),
-				new Point(0.59, -31.89),
-				new Point(-39.09, 20.78),
-				new Point(42.97, -48.98),
-				new Point(34.36, 49.08),
-				new Point(-21.91, -49.01),
-				new Point(-46.68, 46.04),
-				new Point(48.52, -43.67),
-				new Point(30.05, 49.25),
-				new Point(4.03, -43.56),
-				new Point(-37.85, 41.72),
-				new Point(38.24, -48.32),
-				new Point(20.83, 57.85)
-				);
+		List<Point> pointList = new LinkedList<Point>();
+		for (Object[] point : POINTS) {
+			pointList.add(new Point((Double) point[0], (Double) point[1]));
+		}
+		return env.fromCollection(pointList);
 	}
 	
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
index 7881226..b7ee4b5 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/wordcount/util/WordCountData.java
@@ -28,44 +28,45 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  */
 public class WordCountData {
 
+	public static final String[] WORDS = new String[] {
+		"To be, or not to be,--that is the question:--",
+		"Whether 'tis nobler in the mind to suffer",
+		"The slings and arrows of outrageous fortune",
+		"Or to take arms against a sea of troubles,",
+		"And by opposing end them?--To die,--to sleep,--",
+		"No more; and by a sleep to say we end",
+		"The heartache, and the thousand natural shocks",
+		"That flesh is heir to,--'tis a consummation",
+		"Devoutly to be wish'd. To die,--to sleep;--",
+		"To sleep! perchance to dream:--ay, there's the rub;",
+		"For in that sleep of death what dreams may come,",
+		"When we have shuffled off this mortal coil,",
+		"Must give us pause: there's the respect",
+		"That makes calamity of so long life;",
+		"For who would bear the whips and scorns of time,",
+		"The oppressor's wrong, the proud man's contumely,",
+		"The pangs of despis'd love, the law's delay,",
+		"The insolence of office, and the spurns",
+		"That patient merit of the unworthy takes,",
+		"When he himself might his quietus make",
+		"With a bare bodkin? who would these fardels bear,",
+		"To grunt and sweat under a weary life,",
+		"But that the dread of something after death,--",
+		"The undiscover'd country, from whose bourn",
+		"No traveller returns,--puzzles the will,",
+		"And makes us rather bear those ills we have",
+		"Than fly to others that we know not of?",
+		"Thus conscience does make cowards of us all;",
+		"And thus the native hue of resolution",
+		"Is sicklied o'er with the pale cast of thought;",
+		"And enterprises of great pith and moment,",
+		"With this regard, their currents turn awry,",
+		"And lose the name of action.--Soft you now!",
+		"The fair Ophelia!--Nymph, in thy orisons",
+		"Be all my sins remember'd."
+	};
+
 	public static DataSet<String> getDefaultTextLineDataSet(ExecutionEnvironment env) {
-		
-		return env.fromElements(
-				"To be, or not to be,--that is the question:--",
-				"Whether 'tis nobler in the mind to suffer",
-				"The slings and arrows of outrageous fortune",
-				"Or to take arms against a sea of troubles,",
-				"And by opposing end them?--To die,--to sleep,--",
-				"No more; and by a sleep to say we end",
-				"The heartache, and the thousand natural shocks",
-				"That flesh is heir to,--'tis a consummation",
-				"Devoutly to be wish'd. To die,--to sleep;--",
-				"To sleep! perchance to dream:--ay, there's the rub;",
-				"For in that sleep of death what dreams may come,",
-				"When we have shuffled off this mortal coil,",
-				"Must give us pause: there's the respect",
-				"That makes calamity of so long life;",
-				"For who would bear the whips and scorns of time,",
-				"The oppressor's wrong, the proud man's contumely,",
-				"The pangs of despis'd love, the law's delay,",
-				"The insolence of office, and the spurns",
-				"That patient merit of the unworthy takes,",
-				"When he himself might his quietus make",
-				"With a bare bodkin? who would these fardels bear,",
-				"To grunt and sweat under a weary life,",
-				"But that the dread of something after death,--",
-				"The undiscover'd country, from whose bourn",
-				"No traveller returns,--puzzles the will,",
-				"And makes us rather bear those ills we have",
-				"Than fly to others that we know not of?",
-				"Thus conscience does make cowards of us all;",
-				"And thus the native hue of resolution",
-				"Is sicklied o'er with the pale cast of thought;",
-				"And enterprises of great pith and moment,",
-				"With this regard, their currents turn awry,",
-				"And lose the name of action.--Soft you now!",
-				"The fair Ophelia!--Nymph, in thy orisons",
-				"Be all my sins remember'd."
-				);
+		return env.fromElements(WORDS);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/pom.xml b/flink-examples/flink-scala-examples/pom.xml
index 89e5306..a6801f8 100644
--- a/flink-examples/flink-scala-examples/pom.xml
+++ b/flink-examples/flink-scala-examples/pom.xml
@@ -38,6 +38,11 @@ under the License.
 			<artifactId>flink-scala</artifactId>
 			<version>${project.version}</version>
 		</dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-java-examples</artifactId>
+            <version>${project.version}</version>
+        </dependency>
 	</dependencies>
 
 	<build>
@@ -232,7 +237,8 @@ under the License.
 							</includes>
 						</configuration>
 					</execution>
-		
+		               -->
+
 					<execution>
 						<id>WordCount</id>
 						<phase>package</phase>
@@ -254,7 +260,7 @@ under the License.
 							</includes>
 						</configuration>
 					</execution>
-
+                    <!--
 					<execution>
 						<id>ConnectedComponents</id>
 						<phase>package</phase>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
new file mode 100644
index 0000000..b6ec64b
--- /dev/null
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
@@ -0,0 +1,248 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.examples.scala.clustering
+
+import org.apache.flink.api.common.functions._
+import org.apache.flink.api.scala._
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.example.java.clustering.util.KMeansData
+
+import scala.collection.JavaConverters._
+
+/**
+ * This example implements a basic K-Means clustering algorithm.
+ *
+ * K-Means is an iterative clustering algorithm and works as follows:
+ * K-Means is given a set of data points to be clustered and an initial set of ''K'' cluster
+ * centers.
+ * In each iteration, the algorithm computes the distance of each data point to each cluster center.
+ * Each point is assigned to the cluster center which is closest to it.
+ * Subsequently, each cluster center is moved to the center (''mean'') of all points that have
+ * been assigned to it.
+ * The moved cluster centers are fed into the next iteration. 
+ * The algorithm terminates after a fixed number of iterations (as in this implementation) 
+ * or if cluster centers do not (significantly) move in an iteration.
+ * This is the Wikipedia entry for the [[http://en.wikipedia
+ * .org/wiki/K-means_clustering K-Means Clustering algorithm]].
+ *
+ * This implementation works on two-dimensional data points.
+ * It computes an assignment of data points to cluster centers, i.e., 
+ * each data point is annotated with the id of the final cluster (center) it belongs to.
+ *
+ * Input files are plain text files and must be formatted as follows:
+ *
+ *  - Data points are represented as two double values separated by a blank character.
+ *    Data points are separated by newline characters.
+ *    For example `"1.2 2.3\n5.3 7.2\n"` gives two data points (x=1.2, y=2.3) and (x=5.3,
+ *    y=7.2).
+ *  - Cluster centers are represented by an integer id and a point value.
+ *    For example `"1 6.2 3.2\n2 2.9 5.7\n"` gives two centers (id=1, x=6.2,
+ *    y=3.2) and (id=2, x=2.9, y=5.7).
+ *
+ * Usage:
+ * {{{
+ *   KMeans <points path> <centers path> <result path> <num iterations>
+ * }}}
+ * If no parameters are provided, the program is run with default data from `KMeansData`
+ * and 10 iterations.
+ *
+ * This example shows how to use:
+ *
+ *  - Bulk iterations
+ *  - Broadcast variables in bulk iterations
+ *  - Custom Java objects (PoJos)
+ */
+object KMeans {
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    val points: DataSet[Point] = getPointDataSet(env)
+    val centroids: DataSet[Centroid] = getCentroidDataSet(env)
+
+    val finalCentroids = centroids.iterate(numIterations) { currentCentroids =>
+      val newCentroids = points
+        .map(new SelectNearestCenter).withBroadcastSet(currentCentroids, "centroids")
+        .map { x => (x._1, x._2, 1L) }
+        .groupBy(0)
+        .reduce { (p1, p2) => (p1._1, p1._2.add(p2._2), p1._3 + p2._3) }
+        .map { x => new Centroid(x._1, x._2.div(x._3)) }
+      newCentroids
+    }
+
+    val clusteredPoints: DataSet[(Int, Point)] =
+      points.map(new SelectNearestCenter).withBroadcastSet(finalCentroids, "centroids")
+
+    if (fileOutput) {
+      clusteredPoints.writeAsCsv(outputPath, "\n", " ")
+    }
+    else {
+      clusteredPoints.print()
+    }
+
+    env.execute("Scala KMeans Example")
+  }
+
+  private def parseParameters(programArguments: Array[String]): Boolean = {
+    if (programArguments.length > 0) {
+      fileOutput = true
+      if (programArguments.length == 4) {
+        pointsPath = programArguments(0)
+        centersPath = programArguments(1)
+        outputPath = programArguments(2)
+        numIterations = Integer.parseInt(programArguments(3))
+      }
+      else {
+        System.err.println("Usage: KMeans <points path> <centers path> <result path> <num " +
+          "iterations>")
+        false
+      }
+    }
+    else {
+      System.out.println("Executing K-Means example with default parameters and built-in default " +
+        "data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  We provide a data generator to create synthetic input files for this " +
+        "program.")
+      System.out.println("  Usage: KMeans <points path> <centers path> <result path> <num " +
+        "iterations>")
+    }
+    true
+  }
+
+  private def getPointDataSet(env: ExecutionEnvironment): DataSet[Point] = {
+    if (fileOutput) {
+      env.readCsvFile[(Double, Double)](
+        pointsPath,
+        fieldDelimiter = ' ',
+        includedFields = Array(0, 1))
+        .map { x => new Point(x._1, x._2)}
+    }
+    else {
+      val points = KMeansData.POINTS map {
+        case Array(x, y) => new Point(x.asInstanceOf[Double], y.asInstanceOf[Double])
+      }
+      env.fromCollection(points)
+    }
+  }
+
+  private def getCentroidDataSet(env: ExecutionEnvironment): DataSet[Centroid] = {
+    if (fileOutput) {
+      env.readCsvFile[(Int, Double, Double)](
+        centersPath,
+        fieldDelimiter = ' ',
+        includedFields = Array(0, 1, 2))
+        .map { x => new Centroid(x._1, x._2, x._3)}
+    }
+    else {
+      val centroids = KMeansData.CENTROIDS map {
+        case Array(id, x, y) =>
+          new Centroid(id.asInstanceOf[Int], x.asInstanceOf[Double], y.asInstanceOf[Double])
+      }
+      env.fromCollection(centroids)
+    }
+  }
+
+  private var fileOutput: Boolean = false
+  private var pointsPath: String = null
+  private var centersPath: String = null
+  private var outputPath: String = null
+  private var numIterations: Int = 10
+
+  /**
+   * A simple two-dimensional point.
+   */
+  class Point(var x: Double, var y: Double) extends Serializable {
+    def this() {
+      this(0, 0)
+    }
+
+    def add(other: Point): Point = {
+      x += other.x
+      y += other.y
+      this
+    }
+
+    def div(other: Long): Point = {
+      x /= other
+      y /= other
+      this
+    }
+
+    def euclideanDistance(other: Point): Double = {
+      Math.sqrt((x - other.x) * (x - other.x) + (y - other.y) * (y - other.y))
+    }
+
+    def clear(): Unit = {
+      x = 0
+      y = 0
+    }
+
+    override def toString: String = {
+      x + " " + y
+    }
+  }
+
+  /**
+   * A simple two-dimensional centroid, basically a point with an ID.
+   */
+  class Centroid(var id: Int, x: Double, y: Double) extends Point(x, y) {
+    def this() {
+      this(0, 0, 0)
+    }
+
+    def this(id: Int, p: Point) {
+      this(id, p.x, p.y)
+    }
+
+    override def toString: String = {
+      id + " " + super.toString
+    }
+  }
+
+  /** Determines the closest cluster center for a data point. */
+  final class SelectNearestCenter extends RichMapFunction[Point, (Int, Point)] {
+    private var centroids: Traversable[Centroid] = null
+
+    /** Reads the centroid values from a broadcast variable into a collection. */
+    override def open(parameters: Configuration) {
+      centroids = getRuntimeContext.getBroadcastVariable[Centroid]("centroids").asScala
+    }
+
+    def map(p: Point): (Int, Point) = {
+      var minDistance: Double = Double.MaxValue
+      var closestCentroidId: Int = -1
+      for (centroid <- centroids) {
+        val distance = p.euclideanDistance(centroid)
+        if (distance < minDistance) {
+          minDistance = distance
+          closestCentroidId = centroid.id
+        }
+      }
+      (closestCentroidId, p)
+    }
+
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
index 5ec7e54..50f01c9 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/BatchGradientDescent.scala
@@ -1,83 +1,83 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.datamining
-
-import scala.math._
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-
-abstract class BatchGradientDescent(eps: Double, eta: Double, lambda: Double, examplesInput: String, weightsInput: String, weightsOutput: String) extends Serializable {
-  def computeGradient(example: Array[Double], weight: Array[Double]): (Double, Array[Double])
-
-  def updateWeight = (prev: (Int, Array[Double], Double), vg: ValueAndGradient) => {
-    val (id, wOld, eta) = prev
-    val ValueAndGradient(_, lossSum, gradSum) = vg
-
-    val delta = lossSum + lambda * wOld.norm
-    val wNew = (wOld + (gradSum * eta)) * (1 - eta * lambda)
-    (id, delta, wNew, eta * 0.9)
-  }
-
-  class WeightVector(vector: Array[Double]) {
-    def +(that: Array[Double]): Array[Double] = (vector zip that) map { case (x1, x2) => x1 + x2 }
-    def -(that: Array[Double]): Array[Double] = (vector zip that) map { case (x1, x2) => x1 - x2 }
-    def *(x: Double): Array[Double] = vector map { x * _ }
-    def norm: Double = sqrt(vector map { x => x * x } reduce { _ + _ })
-  }
-
-  implicit def array2WeightVector(vector: Array[Double]): WeightVector = new WeightVector(vector)
-
-  case class ValueAndGradient(id: Int, value: Double, gradient: Array[Double]) {
-    def this(id: Int, vg: (Double, Array[Double])) = this(id, vg._1, vg._2)
-    def +(that: ValueAndGradient) = ValueAndGradient(id, value + that.value, gradient + that.gradient)
-  }
-
-  def readVector = (line: String) => {
-    val Seq(id, vector @ _*) = line.split(',').toSeq
-    id.toInt -> (vector map { _.toDouble } toArray)
-  }
-
-  def formatOutput = (id: Int, vector: Array[Double]) => "%s,%s".format(id, vector.mkString(","))
-
-  def getPlan() = {
-
-    val examples = DataSource(examplesInput, DelimitedInputFormat(readVector))
-    val weights = DataSource(weightsInput, DelimitedInputFormat(readVector))
-
-    def gradientDescent = (s: DataSet[(Int, Array[Double])], ws: DataSet[(Int, Array[Double], Double)]) => {
-
-      val lossesAndGradients = ws cross examples map { (w, ex) => new ValueAndGradient(w._1, computeGradient(ex._2, w._2)) }
-      val lossAndGradientSums = lossesAndGradients groupBy { _.id } reduce (_ + _)
-      val newWeights = ws join lossAndGradientSums where { _._1 } isEqualTo { _.id } map updateWeight
-
-      val s1 = newWeights map { case (wId, _, wNew, _) => (wId, wNew) } // updated solution elements
-      val ws1 = newWeights filter { case (_, delta, _, _) => delta > eps } map { case (wId, _, wNew, etaNew) => (wId, wNew, etaNew) } // new workset
-
-      (s1, ws1)
-    }
-
-    val newWeights = weights.iterateWithDelta(weights.map { case (id, w) => (id, w, eta) }, {_._1}, gradientDescent, 10)
-
-    val output = newWeights.write(weightsOutput, DelimitedOutputFormat(formatOutput.tupled))
-    new ScalaPlan(Seq(output), "Batch Gradient Descent")
-  }
-}
-
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.datamining
+//
+//import scala.math._
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//
+//abstract class BatchGradientDescent(eps: Double, eta: Double, lambda: Double, examplesInput: String, weightsInput: String, weightsOutput: String) extends Serializable {
+//  def computeGradient(example: Array[Double], weight: Array[Double]): (Double, Array[Double])
+//
+//  def updateWeight = (prev: (Int, Array[Double], Double), vg: ValueAndGradient) => {
+//    val (id, wOld, eta) = prev
+//    val ValueAndGradient(_, lossSum, gradSum) = vg
+//
+//    val delta = lossSum + lambda * wOld.norm
+//    val wNew = (wOld + (gradSum * eta)) * (1 - eta * lambda)
+//    (id, delta, wNew, eta * 0.9)
+//  }
+//
+//  class WeightVector(vector: Array[Double]) {
+//    def +(that: Array[Double]): Array[Double] = (vector zip that) map { case (x1, x2) => x1 + x2 }
+//    def -(that: Array[Double]): Array[Double] = (vector zip that) map { case (x1, x2) => x1 - x2 }
+//    def *(x: Double): Array[Double] = vector map { x * _ }
+//    def norm: Double = sqrt(vector map { x => x * x } reduce { _ + _ })
+//  }
+//
+//  implicit def array2WeightVector(vector: Array[Double]): WeightVector = new WeightVector(vector)
+//
+//  case class ValueAndGradient(id: Int, value: Double, gradient: Array[Double]) {
+//    def this(id: Int, vg: (Double, Array[Double])) = this(id, vg._1, vg._2)
+//    def +(that: ValueAndGradient) = ValueAndGradient(id, value + that.value, gradient + that.gradient)
+//  }
+//
+//  def readVector = (line: String) => {
+//    val Seq(id, vector @ _*) = line.split(',').toSeq
+//    id.toInt -> (vector map { _.toDouble } toArray)
+//  }
+//
+//  def formatOutput = (id: Int, vector: Array[Double]) => "%s,%s".format(id, vector.mkString(","))
+//
+//  def getPlan() = {
+//
+//    val examples = DataSource(examplesInput, DelimitedInputFormat(readVector))
+//    val weights = DataSource(weightsInput, DelimitedInputFormat(readVector))
+//
+//    def gradientDescent = (s: DataSetOLD[(Int, Array[Double])], ws: DataSetOLD[(Int, Array[Double], Double)]) => {
+//
+//      val lossesAndGradients = ws cross examples map { (w, ex) => new ValueAndGradient(w._1, computeGradient(ex._2, w._2)) }
+//      val lossAndGradientSums = lossesAndGradients groupBy { _.id } reduce (_ + _)
+//      val newWeights = ws join lossAndGradientSums where { _._1 } isEqualTo { _.id } map updateWeight
+//
+//      val s1 = newWeights map { case (wId, _, wNew, _) => (wId, wNew) } // updated solution elements
+//      val ws1 = newWeights filter { case (_, delta, _, _) => delta > eps } map { case (wId, _, wNew, etaNew) => (wId, wNew, etaNew) } // new workset
+//
+//      (s1, ws1)
+//    }
+//
+//    val newWeights = weights.iterateWithDelta(weights.map { case (id, w) => (id, w, eta) }, {_._1}, gradientDescent, 10)
+//
+//    val output = newWeights.write(weightsOutput, DelimitedOutputFormat(formatOutput.tupled))
+//    new ScalaPlan(Seq(output), "Batch Gradient Descent")
+//  }
+//}
+//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/KMeans.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/KMeans.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/KMeans.scala
deleted file mode 100644
index e2248fb..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/datamining/KMeans.scala
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.examples.scala.datamining
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-
-class KMeans extends Program with ProgramDescription with Serializable {
-
-  case class Point(x: Double, y: Double, z: Double) {
-    
-    def +(other: Point) = { Point(x + other.x, y + other.y, z + other.z) }
-    
-    def /(div: Int) = { Point(x / div, y / div, z / div) }
-    
-    def computeEuclidianDistance(other: Point) = {
-      math.sqrt(math.pow(x - other.x, 2) + math.pow(y - other.y, 2) + math.pow(z - other.z, 2))
-    }
-  }
-
-  case class Distance(dataPoint: Point, clusterId: Int, distance: Double)
-  
-  def formatCenterOutput = ((cid: Int, p: Point) => "%d|%.1f|%.1f|%.1f|".format(cid, p.x, p.y, p.z)).tupled
-
-
-  def getScalaPlan(dop: Int, dataPointInput: String, clusterInput: String, clusterOutput: String, numIterations: Int) = {
-    
-    val dataPoints = DataSource(dataPointInput, CsvInputFormat[(Int, Double, Double, Double)]("\n", '|')) 
-                       .map { case (id, x, y, z) => (id, Point(x, y, z)) }
-  
-    val clusterPoints = DataSource(clusterInput, CsvInputFormat[(Int, Double, Double, Double)]("\n", '|'))
-                       .map { case (id, x, y, z) => (id, Point(x, y, z)) }
-
-
-    // iterate the K-Means function, starting with the initial cluster points
-    val finalCenters = clusterPoints.iterate(numIterations, { centers =>
-
-        // compute the distance between each point and all current centroids
-        val distances = dataPoints cross centers map { (point, center) =>
-            val ((pid, dataPoint), (cid, clusterPoint)) = (point, center)
-            val distToCluster = dataPoint.computeEuclidianDistance(clusterPoint)
-            (pid, Distance(dataPoint, cid, distToCluster))
-        }
-      
-        // pick for each point the closest centroid
-        val nearestCenters = distances groupBy { case (pid, _) => pid } reduceGroup { ds => ds.minBy(_._2.distance) }
-        
-        // for each centroid, average among all data points that have chosen it as the closest one
-        // the average is computed as sum, count, finalized as sum/count
-        nearestCenters
-              .map { case (_, Distance(dataPoint, cid, _)) => (cid, dataPoint, 1) }
-              .groupBy {_._1} .reduce { (a, b) => (a._1, a._2 + b._2, a._3 + b._3) }
-              .map { case (cid, centerPoint, num) => (cid, centerPoint / num) }
-    })
-
-    val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatCenterOutput))
-
-    new ScalaPlan(Seq(output), "KMeans Iteration")
-  }
-
-  
-  /**
-   * The program entry point for the packaged version of the program.
-   * 
-   * @param args The command line arguments, including, consisting of the following parameters:
-   *             <numSubStasks> <dataPoints> <clusterCenters> <output> <numIterations>"
-   * @return The program plan of the kmeans example program.
-   */
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4).toInt)
-  }
-    
-  override def getDescription() = {
-    "Parameters: <numSubStasks> <dataPoints> <clusterCenters> <output> <numIterations>"
-  }
-}
-
-/**
- * Entry point to make the example standalone runnable with the local executor
- */
-object RunKMeans {
-
-  def main(args: Array[String]) {
-    val km = new KMeans
-    if (args.size < 5) {
-      println(km.getDescription)
-      return
-    }
-    val plan = km.getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4).toInt)
-    LocalExecutor.execute(plan)
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
index 125ebd6..5200074 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ComputeEdgeDegrees.scala
@@ -1,123 +1,123 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.graph
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-/**
- * Annotates edges with associated vertex degrees.
- */
-class ComputeEdgeDegrees extends Program with ProgramDescription with Serializable {
-  override def getDescription() = {
-    "Parameters: [numSubStasks] [input file] [output file]"
-  }
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2))
-  }
-   
-  /*
-   * Output formatting function for edges with annotated degrees
-   */
-  def formatEdgeWithDegrees = (v1: Int, v2: Int, c1: Int, c2: Int) => "%d,%d|%d,%d".format(v1, v2, c1, c2)
-    
-  /*
-   * Emits one edge for each unique input edge with the vertex degree of the first(and grouping key) vertex.
-   * The degree of the second (non-grouping key) vertexes are set to zero.
-   * Edges are projected such that smaller vertex is the first vertex.
-   */ 
-  def annotateFirstVertexDegree(eI: Iterator[(Int, Int)]): List[(Int, Int, Int, Int)] = {
-    val eL = eI.toList
-    val eLUniq = eL.distinct
-    val cnt = eLUniq.size
-    for (e <- eLUniq)
-      yield if (e._1 < e._2) 
-    	  		(e._1, e._2, cnt, 0)
-        	else 
-        		(e._2, e._1, 0, cnt)
-  }
-  
-  /*
-   * Combines the degrees of both vertexes of an edge.
-   */
-  def combineVertexDegrees(eI: Iterator[(Int, Int, Int, Int)]) : (Int, Int, Int, Int) = {
-    
-    val eL = eI.toList
-    if (eL.size != 2)
-    	throw new RuntimeException("Problem when combinig vertex counts");
-    
-    if (eL(0)._3 == 0 && eL(1)._4 == 0)
-      (eL(0)._1, eL(1)._3, eL(0)._2, eL(0)._4)
-    else
-      (eL(0)._1, eL(0)._3, eL(0)._2, eL(1)._4)
-    
-  }
-    
-  def getScalaPlan(numSubTasks: Int, edgeInput: String, annotatedEdgeOutput: String) = {
-    
-    /*
-     * Input format for edges. 
-     * Edges are separated by new line '\n'. 
-     * An edge is represented as two Integer vertex IDs which are separated by a blank ','.
-     */
-    val edges = DataSource(edgeInput, CsvInputFormat[(Int, Int)]("\n", ','))
-
-    /*
-     * Emit each edge twice with both vertex orders.
-     */
-    val projEdges = edges flatMap { (e) => Iterator((e._1, e._2) , (e._2, e._1)) }
-    
-    /*
-     * Annotates each edges with degree for the first vertex.
-     */
-    val vertexCnts = projEdges groupBy { _._1 } reduceGroup { annotateFirstVertexDegree } flatMap {x => x.iterator }
-    
-    /*
-     * Combines the degrees of both vertexes of an edge.
-     */
-    val combinedVertexCnts = vertexCnts groupBy { (x) => (x._1, x._2) } reduceGroup { combineVertexDegrees }
-    
-    /*
-     * Emit annotated edges.
-     */
-    val output = combinedVertexCnts.write(annotatedEdgeOutput, DelimitedOutputFormat(formatEdgeWithDegrees.tupled))
-  
-    val plan = new ScalaPlan(Seq(output), "Compute Edge Degrees")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-  
-  object RunComputeEdgeDegrees {
-  def main(args: Array[String]) {
-    val ced = new ComputeEdgeDegrees
-    if (args.size < 3) {
-      println(ced.getDescription)
-      return
-    }
-    val plan = ced.getScalaPlan(args(0).toInt, args(1), args(2))
-    LocalExecutor.execute(plan)
-    System.exit(0)
-  }
-}
-}
\ No newline at end of file
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.graph
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+///**
+// * Annotates edges with associated vertex degrees.
+// */
+//class ComputeEdgeDegrees extends Program with ProgramDescription with Serializable {
+//  override def getDescription() = {
+//    "Parameters: [numSubStasks] [input file] [output file]"
+//  }
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1), args(2))
+//  }
+//
+//  /*
+//   * Output formatting function for edges with annotated degrees
+//   */
+//  def formatEdgeWithDegrees = (v1: Int, v2: Int, c1: Int, c2: Int) => "%d,%d|%d,%d".format(v1, v2, c1, c2)
+//
+//  /*
+//   * Emits one edge for each unique input edge with the vertex degree of the first(and grouping key) vertex.
+//   * The degree of the second (non-grouping key) vertexes are set to zero.
+//   * Edges are projected such that smaller vertex is the first vertex.
+//   */
+//  def annotateFirstVertexDegree(eI: Iterator[(Int, Int)]): List[(Int, Int, Int, Int)] = {
+//    val eL = eI.toList
+//    val eLUniq = eL.distinct
+//    val cnt = eLUniq.size
+//    for (e <- eLUniq)
+//      yield if (e._1 < e._2)
+//    	  		(e._1, e._2, cnt, 0)
+//        	else
+//        		(e._2, e._1, 0, cnt)
+//  }
+//
+//  /*
+//   * Combines the degrees of both vertexes of an edge.
+//   */
+//  def combineVertexDegrees(eI: Iterator[(Int, Int, Int, Int)]) : (Int, Int, Int, Int) = {
+//
+//    val eL = eI.toList
+//    if (eL.size != 2)
+//    	throw new RuntimeException("Problem when combinig vertex counts");
+//
+//    if (eL(0)._3 == 0 && eL(1)._4 == 0)
+//      (eL(0)._1, eL(1)._3, eL(0)._2, eL(0)._4)
+//    else
+//      (eL(0)._1, eL(0)._3, eL(0)._2, eL(1)._4)
+//
+//  }
+//
+//  def getScalaPlan(numSubTasks: Int, edgeInput: String, annotatedEdgeOutput: String) = {
+//
+//    /*
+//     * Input format for edges.
+//     * Edges are separated by new line '\n'.
+//     * An edge is represented as two Integer vertex IDs which are separated by a blank ','.
+//     */
+//    val edges = DataSource(edgeInput, CsvInputFormat[(Int, Int)]("\n", ','))
+//
+//    /*
+//     * Emit each edge twice with both vertex orders.
+//     */
+//    val projEdges = edges flatMap { (e) => Iterator((e._1, e._2) , (e._2, e._1)) }
+//
+//    /*
+//     * Annotates each edges with degree for the first vertex.
+//     */
+//    val vertexCnts = projEdges groupBy { _._1 } reduceGroup { annotateFirstVertexDegree } flatMap {x => x.iterator }
+//
+//    /*
+//     * Combines the degrees of both vertexes of an edge.
+//     */
+//    val combinedVertexCnts = vertexCnts groupBy { (x) => (x._1, x._2) } reduceGroup { combineVertexDegrees }
+//
+//    /*
+//     * Emit annotated edges.
+//     */
+//    val output = combinedVertexCnts.write(annotatedEdgeOutput, DelimitedOutputFormat(formatEdgeWithDegrees.tupled))
+//
+//    val plan = new ScalaPlan(Seq(output), "Compute Edge Degrees")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//
+//  object RunComputeEdgeDegrees {
+//  def main(args: Array[String]) {
+//    val ced = new ComputeEdgeDegrees
+//    if (args.size < 3) {
+//      println(ced.getDescription)
+//      return
+//    }
+//    val plan = ced.getScalaPlan(args(0).toInt, args(1), args(2))
+//    LocalExecutor.execute(plan)
+//    System.exit(0)
+//  }
+//}
+//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
index 972e197..696b75f 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
@@ -1,90 +1,90 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.graph;
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-object RunConnectedComponents {
- def main(pArgs: Array[String]) {
-   
-    if (pArgs.size < 5) {
-      println("USAGE: <vertices input file> <edges input file> <output file> <max iterations> <degree of parallelism>")
-      return
-    }
-    val plan = new ConnectedComponents().getPlan(pArgs(0), pArgs(1), pArgs(2), pArgs(3), pArgs(4))
-    LocalExecutor.execute(plan)
-  }
-}
-
-class ConnectedComponents extends Program with Serializable {
-  
-    override def getPlan(args: String*) = {
-      val plan = getScalaPlan(args(0), args(1), args(2), args(3).toInt)
-      plan.setDefaultParallelism(args(4).toInt)
-      plan
-  }
-  
-  def getScalaPlan(verticesInput: String, edgesInput: String, componentsOutput: String, maxIterations: Int) = {
-
-  val vertices = DataSource(verticesInput, DelimitedInputFormat(parseVertex))
-  val directedEdges = DataSource(edgesInput, DelimitedInputFormat(parseEdge))
-
-  val undirectedEdges = directedEdges flatMap { case (from, to) => Seq(from -> to, to -> from) }
-
-    def propagateComponent(s: DataSet[(Int, Int)], ws: DataSet[(Int, Int)]) = {
-
-      val allNeighbors = ws join undirectedEdges where { case (v, _) => v } isEqualTo { case (from, _) => from } map { (w, e) => e._2 -> w._2 }
-      val minNeighbors = allNeighbors groupBy { case (to, _) => to } reduceGroup { cs => cs minBy { _._2 } }
-
-      // updated solution elements == new workset
-      val s1 = s join minNeighbors where { _._1 } isEqualTo { _._1 } flatMap { (n, s) =>
-        (n, s) match {
-          case ((v, cOld), (_, cNew)) if cNew < cOld => Some((v, cNew))
-          case _ => None
-        }
-      }
-//      s1.left preserves({ case (v, _) => v }, { case (v, _) => v })
-      s1.right preserves({ v=>v }, { v=>v })
-
-      (s1, s1)
-    }
-
-    val components = vertices.iterateWithDelta(vertices, { _._1 }, propagateComponent, maxIterations)
-    val output = components.write(componentsOutput, DelimitedOutputFormat(formatOutput.tupled))
-
-    val plan = new ScalaPlan(Seq(output), "Connected Components")
-    plan
-  }
-
-  def parseVertex = (line: String) => { val v = line.toInt; v -> v }
-
-  val EdgeInputPattern = """(\d+) (\d+)""".r
-
-  def parseEdge = (line: String) => line match {
-    case EdgeInputPattern(from, to) => from.toInt -> to.toInt
-  }
-
-  def formatOutput = (vertex: Int, component: Int) => "%d %d".format(vertex, component)
-}
-
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.graph;
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//object RunConnectedComponents {
+// def main(pArgs: Array[String]) {
+//
+//    if (pArgs.size < 5) {
+//      println("USAGE: <vertices input file> <edges input file> <output file> <max iterations> <degree of parallelism>")
+//      return
+//    }
+//    val plan = new ConnectedComponents().getPlan(pArgs(0), pArgs(1), pArgs(2), pArgs(3), pArgs(4))
+//    LocalExecutor.execute(plan)
+//  }
+//}
+//
+//class ConnectedComponents extends Program with Serializable {
+//
+//    override def getPlan(args: String*) = {
+//      val plan = getScalaPlan(args(0), args(1), args(2), args(3).toInt)
+//      plan.setDefaultParallelism(args(4).toInt)
+//      plan
+//  }
+//
+//  def getScalaPlan(verticesInput: String, edgesInput: String, componentsOutput: String, maxIterations: Int) = {
+//
+//  val vertices = DataSource(verticesInput, DelimitedInputFormat(parseVertex))
+//  val directedEdges = DataSource(edgesInput, DelimitedInputFormat(parseEdge))
+//
+//  val undirectedEdges = directedEdges flatMap { case (from, to) => Seq(from -> to, to -> from) }
+//
+//    def propagateComponent(s: DataSetOLD[(Int, Int)], ws: DataSetOLD[(Int, Int)]) = {
+//
+//      val allNeighbors = ws join undirectedEdges where { case (v, _) => v } isEqualTo { case (from, _) => from } map { (w, e) => e._2 -> w._2 }
+//      val minNeighbors = allNeighbors groupBy { case (to, _) => to } reduceGroup { cs => cs minBy { _._2 } }
+//
+//      // updated solution elements == new workset
+//      val s1 = s join minNeighbors where { _._1 } isEqualTo { _._1 } flatMap { (n, s) =>
+//        (n, s) match {
+//          case ((v, cOld), (_, cNew)) if cNew < cOld => Some((v, cNew))
+//          case _ => None
+//        }
+//      }
+////      s1.left preserves({ case (v, _) => v }, { case (v, _) => v })
+//      s1.right preserves({ v=>v }, { v=>v })
+//
+//      (s1, s1)
+//    }
+//
+//    val components = vertices.iterateWithDelta(vertices, { _._1 }, propagateComponent, maxIterations)
+//    val output = components.write(componentsOutput, DelimitedOutputFormat(formatOutput.tupled))
+//
+//    val plan = new ScalaPlan(Seq(output), "Connected Components")
+//    plan
+//  }
+//
+//  def parseVertex = (line: String) => { val v = line.toInt; v -> v }
+//
+//  val EdgeInputPattern = """(\d+) (\d+)""".r
+//
+//  def parseEdge = (line: String) => line match {
+//    case EdgeInputPattern(from, to) => from.toInt -> to.toInt
+//  }
+//
+//  def formatOutput = (vertex: Int, component: Int) => "%d %d".format(vertex, component)
+//}
+//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
index ecf0cd7..cf943be 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOnEdgesWithDegrees.scala
@@ -1,125 +1,125 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.graph
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import scala.math._
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-import org.apache.flink.api.scala.analysis.GlobalSchemaPrinter
-
-object RunEnumTrianglesOnEdgesWithDegrees {
-  def main(args: Array[String]) {
-    val enumTriangles = new EnumTrianglesOnEdgesWithDegrees
-    if (args.size < 3) {
-      println(enumTriangles.getDescription)
-      return
-    }
-    val plan = enumTriangles.getScalaPlan(args(0).toInt, args(1), args(2))
-    LocalExecutor.execute(plan)
-  }
-}
-
-/**
- * Enumerates all triangles build by three connected vertices in a graph.
- * The graph is represented as edges (pairs of vertices) with annotated vertex degrees. * 
- */
-class EnumTrianglesOnEdgesWithDegrees extends Program with ProgramDescription with Serializable {
-  override def getDescription() = {
-    "Parameters: [numSubStasks] [input file] [output file]"
-  }
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2))
-  }
-
-  /*
-   * Output formatting function for triangles.
-   */
-  def formatTriangle = (v1: Int, v2: Int, v3: Int) => "%d,%d,%d".format(v1, v2, v3)
-  
-  /*
-   * Extracts degree information and projects edges such that lower degree vertex comes first.
-   */
-  def projectVertexesWithDegrees(e: (String, String)): (Int, Int) = {
-    val v1 = e._1.split(",")
-    val v2 = e._2.split(",")
-    if (v1(1).toInt <= v2(1).toInt)
-      (v1(0).toInt, v2(0).toInt)
-    else
-      (v2(0).toInt, v1(0).toInt)
-  } 
-  
-  /*
-   * Joins projected edges on lower vertex id.
-   * Emits a triad (triangle candidate with one missing edge) for each unique combination of edges.
-   * Ensures that vertex 2 and 3 are ordered by vertex id.  
-   */
-  def buildTriads(eI : Iterator[(Int, Int)]): List[(Int, Int, Int)] = {
-    val eL = eI.toList
-    for (e1 <- eL; 
-         e2 <- eL 
-         if e1._2 < e2._2) yield
-      (e1._1, e1._2, e2._2)
-  }
-  
-  def getScalaPlan(numSubTasks: Int, edgeInput: String, triangleOutput: String) = {
-    
-    /*
-     * Input format for edges with degrees
-     * Edges are separated by new line '\n'. 
-     * An edge is represented by two vertex IDs with associated vertex degrees.
-     * The format of an edge is "<vertexID1>,<vertexDegree1>|<vertexID2>,<vertexDegree2>" 
-     */
-    val vertexesWithDegrees = DataSource(edgeInput, CsvInputFormat[(String, String)]("\n", '|'))
-
-    /*
-     * Project edges such that vertex with lower degree comes first (record position 1) and remove the degrees.
-     */
-    val edgesByDegree = vertexesWithDegrees map { projectVertexesWithDegrees }
-    
-    /*
-     * Project edges such that vertex with lower ID comes first (record position) and remove degrees.
-     */
-    val edgesByID = edgesByDegree map { (x) => if (x._1 < x._2) (x._1, x._2) else (x._2, x._1) }
-    
-    /*
-     * Build triads by joining edges on common vertex.
-     */   
-    val triads = edgesByDegree groupBy { _._1 } reduceGroup { buildTriads } flatMap {x => x.iterator }
-    
-    /*
-     * Join triads with projected edges to 'close' triads.
-     * This filters triads without a closing edge.
-     */
-    val triangles = triads join edgesByID where { t => (t._2, t._3) } isEqualTo { e => (e._1, e._2) } map { (t, e) => t }
-    
-    /*
-     * Emit triangles
-     */
-    val output = triangles.write(triangleOutput, DelimitedOutputFormat(formatTriangle.tupled))
-  
-    val plan = new ScalaPlan(Seq(output), "Enumerate Triangles on Edges with Degrees")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-}
\ No newline at end of file
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.graph
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import scala.math._
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//import org.apache.flink.api.scala.analysis.GlobalSchemaPrinter
+//
+//object RunEnumTrianglesOnEdgesWithDegrees {
+//  def main(args: Array[String]) {
+//    val enumTriangles = new EnumTrianglesOnEdgesWithDegrees
+//    if (args.size < 3) {
+//      println(enumTriangles.getDescription)
+//      return
+//    }
+//    val plan = enumTriangles.getScalaPlan(args(0).toInt, args(1), args(2))
+//    LocalExecutor.execute(plan)
+//  }
+//}
+//
+///**
+// * Enumerates all triangles build by three connected vertices in a graph.
+// * The graph is represented as edges (pairs of vertices) with annotated vertex degrees. *
+// */
+//class EnumTrianglesOnEdgesWithDegrees extends Program with ProgramDescription with Serializable {
+//  override def getDescription() = {
+//    "Parameters: [numSubStasks] [input file] [output file]"
+//  }
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1), args(2))
+//  }
+//
+//  /*
+//   * Output formatting function for triangles.
+//   */
+//  def formatTriangle = (v1: Int, v2: Int, v3: Int) => "%d,%d,%d".format(v1, v2, v3)
+//
+//  /*
+//   * Extracts degree information and projects edges such that lower degree vertex comes first.
+//   */
+//  def projectVertexesWithDegrees(e: (String, String)): (Int, Int) = {
+//    val v1 = e._1.split(",")
+//    val v2 = e._2.split(",")
+//    if (v1(1).toInt <= v2(1).toInt)
+//      (v1(0).toInt, v2(0).toInt)
+//    else
+//      (v2(0).toInt, v1(0).toInt)
+//  }
+//
+//  /*
+//   * Joins projected edges on lower vertex id.
+//   * Emits a triad (triangle candidate with one missing edge) for each unique combination of edges.
+//   * Ensures that vertex 2 and 3 are ordered by vertex id.
+//   */
+//  def buildTriads(eI : Iterator[(Int, Int)]): List[(Int, Int, Int)] = {
+//    val eL = eI.toList
+//    for (e1 <- eL;
+//         e2 <- eL
+//         if e1._2 < e2._2) yield
+//      (e1._1, e1._2, e2._2)
+//  }
+//
+//  def getScalaPlan(numSubTasks: Int, edgeInput: String, triangleOutput: String) = {
+//
+//    /*
+//     * Input format for edges with degrees
+//     * Edges are separated by new line '\n'.
+//     * An edge is represented by two vertex IDs with associated vertex degrees.
+//     * The format of an edge is "<vertexID1>,<vertexDegree1>|<vertexID2>,<vertexDegree2>"
+//     */
+//    val vertexesWithDegrees = DataSource(edgeInput, CsvInputFormat[(String, String)]("\n", '|'))
+//
+//    /*
+//     * Project edges such that vertex with lower degree comes first (record position 1) and remove the degrees.
+//     */
+//    val edgesByDegree = vertexesWithDegrees map { projectVertexesWithDegrees }
+//
+//    /*
+//     * Project edges such that vertex with lower ID comes first (record position) and remove degrees.
+//     */
+//    val edgesByID = edgesByDegree map { (x) => if (x._1 < x._2) (x._1, x._2) else (x._2, x._1) }
+//
+//    /*
+//     * Build triads by joining edges on common vertex.
+//     */
+//    val triads = edgesByDegree groupBy { _._1 } reduceGroup { buildTriads } flatMap {x => x.iterator }
+//
+//    /*
+//     * Join triads with projected edges to 'close' triads.
+//     * This filters triads without a closing edge.
+//     */
+//    val triangles = triads join edgesByID where { t => (t._2, t._3) } isEqualTo { e => (e._1, e._2) } map { (t, e) => t }
+//
+//    /*
+//     * Emit triangles
+//     */
+//    val output = triangles.write(triangleOutput, DelimitedOutputFormat(formatTriangle.tupled))
+//
+//    val plan = new ScalaPlan(Seq(output), "Enumerate Triangles on Edges with Degrees")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
index 1d4657d..6902a6f 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
@@ -1,98 +1,96 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.examples.scala.graph
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.{ ProgramDescription, Program }
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.DataSource
-import org.apache.flink.api.scala.operators._
-import org.apache.flink.api.scala.operators.CsvInputFormat
-
-
-class LineRank extends Program with Serializable {
-  
-  case class Edge(source: Int, target: Int, weight: Double)
-  case class VectorElement(index: Int, value: Double)
-
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2), args(3).toInt, args(4))
-  }
-
-  def sumElements(elem1: VectorElement, elem2: VectorElement) = VectorElement(elem1.index, elem1.value + elem2.value)
-
-  def sgtTimes(SGT: DataSet[Edge], vector: DataSet[VectorElement]) = {
-    SGT.join(vector).where(_.source).isEqualTo(_.index)
-      .map((edge, elem) => VectorElement(edge.target, edge.weight * elem.value))
-      .groupBy(_.index).reduce(sumElements)
-  }
-
-  def tgTimes(TG: DataSet[Edge], vector: DataSet[VectorElement]) = {
-    TG.join(vector).where(_.target).isEqualTo(_.index)
-      .map((edge, elem) => VectorElement(edge.source, edge.weight * elem.value))
-  }
-
-  def rescale(v3: DataSet[VectorElement], c: Double, r: Double) = {
-    v3.map(elem => { VectorElement(elem.index, c * elem.value + (1 - c) * r) })
-  }
-
-  def powerMethod(SGT: DataSet[Edge], TG: DataSet[Edge], d: DataSet[VectorElement], c: Double, r: Double)(v: DataSet[VectorElement]) = {
-
-    val v1 = d.join(v).where(_.index).isEqualTo(_.index)
-      .map((leftElem, rightElem) => VectorElement(leftElem.index, leftElem.value * rightElem.value))
-
-    val v2 = sgtTimes(SGT, v1)
-    val v3 = tgTimes(TG, v2)
-    val nextV = rescale(v3, c, r)
-
-    nextV
-  }
-
-  def getScalaPlan(numSubTasks: Int, sourceIncidenceMatrixPath: String, targetIncidenceMatrixPath: String, m: Int,
-    outputPath: String) = {
-
-    val c = .85
-    val r = 1.0 / m
-
-    val SGT = DataSource(sourceIncidenceMatrixPath, CsvInputFormat[Edge]())
-    val TG = DataSource(targetIncidenceMatrixPath, CsvInputFormat[Edge]())
-
-    val d1 = SGT.map(edge => VectorElement(edge.target, edge.weight))
-      .groupBy(_.index)
-      .reduce(sumElements)
-
-    val d2 = tgTimes(TG, d1)
-
-    val d = d2.map(elem => VectorElement(elem.index, 1 / elem.value))
-
-    val initialV1 = d.map(elem => VectorElement(elem.index, elem.value * m))
-    val initialV2 = sgtTimes(SGT, initialV1)
-    val initialV3 = tgTimes(TG, initialV2)
-    val initialV = rescale(initialV3, c, r)
-
-    val v = initialV.iterate(5, powerMethod(SGT, TG, d, c, r))
-
-    val output = v.write(outputPath, CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output), "LineRank")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-}
\ No newline at end of file
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.flink.examples.scala.graph
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.{ ProgramDescription, Program }
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//
+//class LineRank extends Program with Serializable {
+//
+//  case class Edge(source: Int, target: Int, weight: Double)
+//  case class VectorElement(index: Int, value: Double)
+//
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1), args(2), args(3).toInt, args(4))
+//  }
+//
+//  def sumElements(elem1: VectorElement, elem2: VectorElement) = VectorElement(elem1.index, elem1.value + elem2.value)
+//
+//  def sgtTimes(SGT: DataSetOLD[Edge], vector: DataSetOLD[VectorElement]) = {
+//    SGT.join(vector).where(_.source).isEqualTo(_.index)
+//      .map((edge, elem) => VectorElement(edge.target, edge.weight * elem.value))
+//      .groupBy(_.index).reduce(sumElements)
+//  }
+//
+//  def tgTimes(TG: DataSetOLD[Edge], vector: DataSetOLD[VectorElement]) = {
+//    TG.join(vector).where(_.target).isEqualTo(_.index)
+//      .map((edge, elem) => VectorElement(edge.source, edge.weight * elem.value))
+//  }
+//
+//  def rescale(v3: DataSetOLD[VectorElement], c: Double, r: Double) = {
+//    v3.map(elem => { VectorElement(elem.index, c * elem.value + (1 - c) * r) })
+//  }
+//
+//  def powerMethod(SGT: DataSetOLD[Edge], TG: DataSetOLD[Edge], d: DataSetOLD[VectorElement], c: Double, r: Double)(v: DataSetOLD[VectorElement]) = {
+//
+//    val v1 = d.join(v).where(_.index).isEqualTo(_.index)
+//      .map((leftElem, rightElem) => VectorElement(leftElem.index, leftElem.value * rightElem.value))
+//
+//    val v2 = sgtTimes(SGT, v1)
+//    val v3 = tgTimes(TG, v2)
+//    val nextV = rescale(v3, c, r)
+//
+//    nextV
+//  }
+//
+//  def getScalaPlan(numSubTasks: Int, sourceIncidenceMatrixPath: String, targetIncidenceMatrixPath: String, m: Int,
+//    outputPath: String) = {
+//
+//    val c = .85
+//    val r = 1.0 / m
+//
+//    val SGT = DataSource(sourceIncidenceMatrixPath, CsvInputFormat[Edge]())
+//    val TG = DataSource(targetIncidenceMatrixPath, CsvInputFormat[Edge]())
+//
+//    val d1 = SGT.map(edge => VectorElement(edge.target, edge.weight))
+//      .groupBy(_.index)
+//      .reduce(sumElements)
+//
+//    val d2 = tgTimes(TG, d1)
+//
+//    val d = d2.map(elem => VectorElement(elem.index, 1 / elem.value))
+//
+//    val initialV1 = d.map(elem => VectorElement(elem.index, elem.value * m))
+//    val initialV2 = sgtTimes(SGT, initialV1)
+//    val initialV3 = tgTimes(TG, initialV2)
+//    val initialV = rescale(initialV3, c, r)
+//
+//    val v = initialV.iterate(5, powerMethod(SGT, TG, d, c, r))
+//
+//    val output = v.write(outputPath, CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output), "LineRank")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//}
\ No newline at end of file


[36/60] git commit: Pi estimation example job in Scala

Posted by al...@apache.org.
Pi estimation example job in Scala


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

Branch: refs/heads/master
Commit: 299cef7586195f0a8ff26ab70a46cbf5e0f229db
Parents: 0dc7614
Author: Kostas Tzoumas <ko...@kostass-mbp.fritz.box>
Authored: Thu Sep 11 18:25:40 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../examples/scala/misc/PiEstimation.scala      | 53 ++++++++++++++++++++
 1 file changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/299cef75/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
new file mode 100644
index 0000000..d702f61
--- /dev/null
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.examples.scala.misc
+
+import org.apache.flink.api.scala._
+
+object PiEstimation {
+
+  def main(args: Array[String]) {
+
+    val numSamples: Long = if (args.length > 0) args(0).toLong else 1000000
+
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+
+    // count how many of the samples would randomly fall into
+    // the unit circle
+    val count =
+      env.generateSequence(1, numSamples)
+      .map (sample => {
+        val x = Math.random()
+        val y = Math.random()
+        if (x * x + y * y < 1) 1L else 0L
+      })
+      .reduce(_+_)
+
+    // the ratio of the unit circle surface to 4 times the unit square is pi
+    val pi = count
+      .map (_ * 4.0 / numSamples)
+
+    println("We estimate Pi to be:")
+
+    pi.print()
+
+    env.execute("PiEstimation example")
+  }
+
+}


[04/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ClosureCleaner.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ClosureCleaner.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ClosureCleaner.scala
deleted file mode 100644
index 228d87c..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ClosureCleaner.scala
+++ /dev/null
@@ -1,227 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/*
- * NOTE: The code in this file is based on code from the Apache Spark
- * project, licensed under the Apache License v 2.0
- */
-
-package org.apache.flink.api.scala.operators
-
-import java.lang.reflect.Field
-
-import scala.collection.mutable.Map
-import scala.collection.mutable.Set
-
-import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type}
-import org.objectweb.asm.Opcodes._
-import java.io._
-import java.lang.reflect.Modifier
-
-object ClosureCleaner {
-  // Get an ASM class reader for a given class from the JAR that loaded it
-  private def getClassReader(cls: Class[_]): ClassReader = {
-    // Copy data over, before delegating to ClassReader - else we can run out of open file handles.
-    val className = cls.getName.replaceFirst("^.*\\.", "") + ".class"
-    val resourceStream = cls.getResourceAsStream(className)
-    // todo: Fixme - continuing with earlier behavior ...
-    if (resourceStream == null) return new ClassReader(resourceStream)
-
-    val baos = new ByteArrayOutputStream(128)
-    copyStream(resourceStream, baos, true)
-    new ClassReader(new ByteArrayInputStream(baos.toByteArray))
-  }
-
-  // Check whether a class represents a Scala closure
-  private def isClosure(cls: Class[_]): Boolean = {
-    cls.getName.contains("$anonfun$")
-  }
-
-  // Get a list of the classes of the outer objects of a given closure object, obj;
-  // the outer objects are defined as any closures that obj is nested within, plus
-  // possibly the class that the outermost closure is in, if any. We stop searching
-  // for outer objects beyond that because cloning the user's object is probably
-  // not a good idea (whereas we can clone closure objects just fine since we
-  // understand how all their fields are used).
-  private def getOuterClasses(obj: AnyRef): List[Class[_]] = {
-    for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") {
-      f.setAccessible(true)
-      if (isClosure(f.getType)) {
-        return f.getType :: getOuterClasses(f.get(obj))
-      } else {
-        return f.getType :: Nil // Stop at the first $outer that is not a closure
-      }
-    }
-    return Nil
-  }
-
-  // Get a list of the outer objects for a given closure object.
-  private def getOuterObjects(obj: AnyRef): List[AnyRef] = {
-    for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") {
-      f.setAccessible(true)
-      if (isClosure(f.getType)) {
-        return f.get(obj) :: getOuterObjects(f.get(obj))
-      } else {
-        return f.get(obj) :: Nil // Stop at the first $outer that is not a closure
-      }
-    }
-    return Nil
-  }
-
-  private def getInnerClasses(obj: AnyRef): List[Class[_]] = {
-    val seen = Set[Class[_]](obj.getClass)
-    var stack = List[Class[_]](obj.getClass)
-    while (!stack.isEmpty) {
-      val cr = getClassReader(stack.head)
-      stack = stack.tail
-      val set = Set[Class[_]]()
-      cr.accept(new InnerClosureFinder(set), 0)
-      for (cls <- set -- seen) {
-        seen += cls
-        stack = cls :: stack
-      }
-    }
-    return (seen - obj.getClass).toList
-  }
-
-  private def createNullValue(cls: Class[_]): AnyRef = {
-    if (cls.isPrimitive) {
-      new java.lang.Byte(0: Byte) // Should be convertible to any primitive type
-    } else {
-      null
-    }
-  }
-
-  def clean[F <: AnyRef](func: F): F = {
-    // TODO: cache outerClasses / innerClasses / accessedFields
-    val outerClasses = getOuterClasses(func)
-    val innerClasses = getInnerClasses(func)
-    val outerObjects = getOuterObjects(func)
-    val accessedFields = Map[Class[_], Set[String]]()
-    for (cls <- func.getClass :: outerClasses)
-      accessedFields(cls) = Set[String]()
-    for (cls <- func.getClass :: innerClasses)
-      getClassReader(cls).accept(new FieldAccessFinder(accessedFields), 0)
-
-    // Nullify all the fields that are not used, keep $outer, though.
-    // Also, do not mess with the first outer that is not a closure, this
-    // is mostly the outer class/object of the user.
-    var outer: AnyRef = func
-    while (outer != null && isClosure(outer.getClass)) {
-      //      outer = instantiateClass(cls, outer, inInterpreter)
-      val cls = outer.getClass
-      val newOuter = cls.getDeclaredFields.toList.find( _.getName == "$outer")
-        .map { f => f.setAccessible(true); if (f.get(outer) != null) f.get(outer) else null }
-        .getOrElse(null)
-
-      for (field <- cls.getDeclaredFields if !accessedFields(cls).contains(field.getName) &&
-        !Modifier.isStatic(field.getModifiers) && field.getName != "$outer") {
-        field.setAccessible(true)
-        if (field.get(outer) != null) {
-          field.set(outer, null)
-        }
-      }
-//      // when newOuter is the first non-closure, remove the outer pointer,
-//      // this would mostly be the outer user code object/class
-//      if (newOuter != null && !isClosure(newOuter.getClass)) {
-//        for (field <- cls.getDeclaredFields if !accessedFields(cls).contains(field.getName) &&
-//          !Modifier.isStatic(field.getModifiers)) {
-//          field.setAccessible(true)
-//          if (field.get(outer) != null) {
-//            field.set(outer, null)
-//          }
-//        }
-//      }
-      outer = newOuter
-    }
-    func
-  }
-
-  /** Copy all data from an InputStream to an OutputStream */
-  def copyStream(in: InputStream,
-                 out: OutputStream,
-                 closeStreams: Boolean = false)
-  {
-    val buf = new Array[Byte](8192)
-    var n = 0
-    while (n != -1) {
-      n = in.read(buf)
-      if (n != -1) {
-        out.write(buf, 0, n)
-      }
-    }
-    if (closeStreams) {
-      in.close()
-      out.close()
-    }
-  }
-}
-
-class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) {
-  override def visitMethod(access: Int, name: String, desc: String,
-                           sig: String, exceptions: Array[String]): MethodVisitor = {
-    return new MethodVisitor(ASM4) {
-      override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) {
-        if (op == GETFIELD) {
-          for (cl <- output.keys if cl.getName == owner.replace('/', '.')) {
-            output(cl) += name
-          }
-        }
-      }
-
-      override def visitMethodInsn(op: Int, owner: String, name: String,
-                                   desc: String) {
-        // Check for calls a getter method for a variable in an interpreter wrapper object.
-        // This means that the corresponding field will be accessed, so we should save it.
-        if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer")) {
-          for (cl <- output.keys if cl.getName == owner.replace('/', '.')) {
-            output(cl) += name
-          }
-        }
-      }
-    }
-  }
-}
-
-class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) {
-  var myName: String = null
-
-  override def visit(version: Int, access: Int, name: String, sig: String,
-                     superName: String, interfaces: Array[String]) {
-    myName = name
-  }
-
-  override def visitMethod(access: Int, name: String, desc: String,
-                           sig: String, exceptions: Array[String]): MethodVisitor = {
-    return new MethodVisitor(ASM4) {
-      override def visitMethodInsn(op: Int, owner: String, name: String,
-                                   desc: String) {
-        val argTypes = Type.getArgumentTypes(desc)
-        if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0
-          && argTypes(0).toString.startsWith("L") // is it an object?
-          && argTypes(0).getInternalName == myName) {
-          output += Class.forName(
-            owner.replace('/', '.'),
-            false,
-            Thread.currentThread.getContextClassLoader)
-        }
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala
deleted file mode 100644
index 6a71bb7..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CoGroupOperator.scala
+++ /dev/null
@@ -1,242 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import java.util.{ Iterator => JIterator }
-
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.java.record.operators.CoGroupOperator
-import org.apache.flink.api.java.record.functions.{CoGroupFunction => JCoGroupFunction}
-import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper
-import org.apache.flink.configuration.Configuration
-
-import org.apache.flink.api.scala.codegen.{MacroContextHolder, Util}
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.functions.DeserializingIterator
-import org.apache.flink.api.scala.functions.{CoGroupFunctionBase, CoGroupFunction, FlatCoGroupFunction}
-import org.apache.flink.api.scala.analysis.FieldSelector
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.TwoInputHintable
-
-class CoGroupDataSet[LeftIn, RightIn](val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def where[Key](keyFun: LeftIn => Key): CoGroupDataSetWithWhere[LeftIn, RightIn, Key] = macro CoGroupMacros.whereImpl[LeftIn, RightIn, Key]
-}
-
-class CoGroupDataSetWithWhere[LeftIn, RightIn, Key](val leftKeySelection: List[Int], val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def isEqualTo[Key](keyFun: RightIn => Key): CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn] = macro CoGroupMacros.isEqualToImpl[LeftIn, RightIn, Key]
-}
-
-class CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn](val leftKeySelection: List[Int], val rightKeySelection: List[Int], val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def map[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Out): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CoGroupMacros.map[LeftIn, RightIn, Out]
-  def flatMap[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Iterator[Out]): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CoGroupMacros.flatMap[LeftIn, RightIn, Out]
-}
-
-class NoKeyCoGroupBuilder(s: JCoGroupFunction) extends CoGroupOperator.Builder(new UserCodeObjectWrapper(new CoGroupOperator.WrappingCoGroupFunction(s)))
-
-object CoGroupMacros {
-  
-  def whereImpl[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Key: c.WeakTypeTag](c: Context { type PrefixType = CoGroupDataSet[LeftIn, RightIn] })
-                                                                                  (keyFun: c.Expr[LeftIn => Key]): c.Expr[CoGroupDataSetWithWhere[LeftIn, RightIn, Key]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val keySelector = slave.getSelector(keyFun)
-
-    val helper = reify {
-      val helper = c.prefix.splice
-      new CoGroupDataSetWithWhere[LeftIn, RightIn, Key](keySelector.splice, helper.leftInput, helper.rightInput)
-    }
-
-    return helper
-  }
-  
-  def isEqualToImpl[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Key: c.WeakTypeTag](c: Context { type PrefixType = CoGroupDataSetWithWhere[LeftIn, RightIn, Key] })
-                                                                                      (keyFun: c.Expr[RightIn => Key]): c.Expr[CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val keySelector = slave.getSelector(keyFun)
-
-    val helper = reify {
-      val helper = c.prefix.splice
-      new CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn](helper.leftKeySelection, keySelector.splice, helper.leftInput, helper.rightInput)
-    }
-
-    return helper
-  }
-
-  def map[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn] })
-                                                                            (fun: c.Expr[(Iterator[LeftIn], Iterator[RightIn]) => Out]): c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-
-    val stub: c.Expr[CoGroupFunctionBase[LeftIn, RightIn, Out]] = if (fun.actualType <:< weakTypeOf[CoGroupFunction[LeftIn, RightIn, Out]])
-      reify { fun.splice.asInstanceOf[CoGroupFunctionBase[LeftIn, RightIn, Out]] }
-    else reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-      
-      new CoGroupFunctionBase[LeftIn, RightIn, Out] {
-        
-        override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = {
-
-          val firstLeftRecord = leftIterator.initialize(leftRecords)
-          val firstRightRecord = rightIterator.initialize(rightRecords)
-
-          if (firstRightRecord != null) {
-            outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo)
-          }
-          if (firstLeftRecord != null) {
-            outputRecord.copyFrom(firstLeftRecord, leftForwardFrom, leftForwardTo)
-          }
-
-          val output = fun.splice.apply(leftIterator, rightIterator)
-
-          serializer.serialize(output, outputRecord)
-          out.collect(outputRecord)
-        }
-      }
-    }
-    val contract = reify {
-      val helper: CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val leftKeySelector = new FieldSelector(generatedStub.leftInputUDT, helper.leftKeySelection)
-      val rightKeySelector = new FieldSelector(generatedStub.rightInputUDT, helper.rightKeySelection)
-
-      val builder = new NoKeyCoGroupBuilder(generatedStub).input1(leftInput).input2(rightInput)
-
-      val leftKeyPositions = leftKeySelector.selectedFields.toIndexArray
-      val rightKeyPositions = rightKeySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.leftInputUDT.getKeySet(leftKeyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), leftKeyPositions(i), rightKeyPositions(i)) }
-      
-      
-      val ret = new CoGroupOperator(builder) with TwoInputKeyedScalaOperator[LeftIn, RightIn, Out] {
-        override val leftKey: FieldSelector = leftKeySelector
-        override val rightKey: FieldSelector = rightKeySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-      }
-      new DataSet[Out](ret) with TwoInputHintable[LeftIn, RightIn, Out] {}
-    }
-    
-    val result = c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-  def flatMap[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn] })
-                                                                                (fun: c.Expr[(Iterator[LeftIn], Iterator[RightIn]) => Iterator[Out]]): c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]] = {
-     import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-
-    val stub: c.Expr[CoGroupFunctionBase[LeftIn, RightIn, Out]] = if (fun.actualType <:< weakTypeOf[CoGroupFunction[LeftIn, RightIn, Out]])
-      reify { fun.splice.asInstanceOf[CoGroupFunctionBase[LeftIn, RightIn, Out]] }
-    else reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-      
-      new CoGroupFunctionBase[LeftIn, RightIn, Out] {
-        
-        override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = {
-          val firstLeftRecord = leftIterator.initialize(leftRecords)
-          val firstRightRecord = rightIterator.initialize(rightRecords)
-
-          if (firstRightRecord != null) {
-            outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo)
-          }
-          if (firstLeftRecord != null) {
-            outputRecord.copyFrom(firstLeftRecord, leftForwardFrom, leftForwardTo)
-          }
-
-          val output = fun.splice.apply(leftIterator, rightIterator)
-
-          if (output.nonEmpty) {
-
-            for (item <- output) {
-              serializer.serialize(item, outputRecord)
-              out.collect(outputRecord)
-            }
-          }
-        }
-      }
-    }
-    val contract = reify {
-      val helper: CoGroupDataSetWithWhereAndEqual[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val leftKeySelector = new FieldSelector(generatedStub.leftInputUDT, helper.leftKeySelection)
-      val rightKeySelector = new FieldSelector(generatedStub.rightInputUDT, helper.rightKeySelection)
-
-      val builder = new NoKeyCoGroupBuilder(generatedStub).input1(leftInput).input2(rightInput)
-
-      val leftKeyPositions = leftKeySelector.selectedFields.toIndexArray
-      val rightKeyPositions = rightKeySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.leftInputUDT.getKeySet(leftKeyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), leftKeyPositions(i), rightKeyPositions(i)) }
-      
-      
-      val ret = new CoGroupOperator(builder) with TwoInputKeyedScalaOperator[LeftIn, RightIn, Out] {
-        override val leftKey: FieldSelector = leftKeySelector
-        override val rightKey: FieldSelector = rightKeySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-      }
-      new DataSet[Out](ret) with TwoInputHintable[LeftIn, RightIn, Out] {}
-    }
-    
-    val result = c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CopyOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CopyOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CopyOperator.scala
deleted file mode 100644
index fd02d8d..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CopyOperator.scala
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import scala.language.reflectiveCalls
-
-import org.apache.flink.api.scala.ScalaOperator
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.OneInputScalaOperator
-import org.apache.flink.api.scala.analysis.UDT
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.analysis.UDTSerializer
-
-import org.apache.flink.util.Collector
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.types.Record
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.java.record.functions.MapFunction
-import org.apache.flink.api.java.record.operators.MapOperator
-
-object CopyOperator {
-  def apply(source: Operator[Record] with ScalaOperator[_, _]): DataSet[_] = {
-    val generatedStub = new MapFunction with Serializable {
-      val udf: UDF1[_, _] = new UDF1(source.getUDF.outputUDT, source.getUDF.outputUDT)
-
-      private var from: Array[Int] = _
-      private var to: Array[Int] = _
-      private var discard: Array[Int] = _
-      private var outputLength: Int = _
-
-      override def open(config: Configuration) = {
-        super.open(config)
-        this.from = udf.inputFields.toSerializerIndexArray
-        this.to = udf.outputFields.toSerializerIndexArray
-        this.discard = udf.getDiscardIndexArray.filter(_ < udf.getOutputLength)
-        this.outputLength = udf.getOutputLength
-      }
-
-      override def map(record: Record, out: Collector[Record]) = {
-
-        record.setNumFields(outputLength)
-
-        record.copyFrom(record, from, to)
-
-        for (field <- discard)
-          record.setNull(field)
-
-        out.collect(record)
-      }
-    }
-
-    val builder = MapOperator.builder(generatedStub).input(source)
-
-    val ret = new MapOperator(builder) with OneInputScalaOperator[Nothing, Nothing] {
-      override def getUDF = generatedStub.udf.asInstanceOf[UDF1[Nothing, Nothing]]
-      override def annotations = Seq(Annotations.getConstantFields(
-        generatedStub.udf.getForwardIndexArrayFrom.zip(generatedStub.udf.getForwardIndexArrayTo)
-          .filter( z => z._1 == z._2).map { _._1}))
-      persistHints = { () =>
-        this.setName("Copy " + source.getName())
-        if (source.getCompilerHints().getAvgOutputRecordSize() >= 0)
-          this.getCompilerHints().setAvgOutputRecordSize(source.getCompilerHints().getAvgOutputRecordSize())
-      }
-    }
-    new DataSet[Nothing](ret)
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala
deleted file mode 100644
index aa04093..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/CrossOperator.scala
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-
-import scala.reflect.macros.Context
-
-import java.util.{ Iterator => JIterator }
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.functions.{CrossFunctionBase, CrossFunction}//, FlatCrossFunction}
-import org.apache.flink.api.scala.codegen.{MacroContextHolder, Util}
-import org.apache.flink.api.scala.functions.DeserializingIterator
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.TwoInputHintable
-
-import org.apache.flink.api.java.record.operators.MapOperator
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.java.record.operators.CrossOperator
-import org.apache.flink.configuration.Configuration
-
-class CrossDataSet[LeftIn, RightIn](val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CrossMacros.map[LeftIn, RightIn, Out]
-  //def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro CrossMacros.flatMap[LeftIn, RightIn, Out]
-  def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)] = macro CrossMacros.filter[LeftIn, RightIn]
-}
-
-object CrossMacros {
-
-  def map[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = CrossDataSet[LeftIn, RightIn] })
-                                                                            (fun: c.Expr[(LeftIn, RightIn) => Out]): c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-
-    val stub: c.Expr[CrossFunctionBase[LeftIn, RightIn, Out]] = if (fun.actualType <:< weakTypeOf[CrossFunction[LeftIn, RightIn, Out]])
-      reify { fun.splice.asInstanceOf[CrossFunctionBase[LeftIn, RightIn, Out]] }
-    else reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-      new CrossFunctionBase[LeftIn, RightIn, Out] {
-        override def cross(leftRecord: Record, rightRecord: Record) : Record = {
-          val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-          val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-          val output = fun.splice.apply(left, right)
-
-          leftRecord.setNumFields(outputLength)
-
-          for (field <- leftDiscard)
-            leftRecord.setNull(field)
-
-          leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo)
-          leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo)
-
-          serializer.serialize(output, leftRecord)
-          leftRecord
-        }
-      }
-    }
-    val contract = reify {
-      val helper: CrossDataSet[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val builder = CrossOperator.builder(generatedStub).input1(leftInput).input2(rightInput)
-      
-      val ret = new CrossOperator(builder) with TwoInputScalaOperator[LeftIn, RightIn, Out] {
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-      }
-      new DataSet[Out](ret) with TwoInputHintable[LeftIn, RightIn, Out] {}
-    }
-
-    val result = c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-
-  def filter[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag](c: Context { type PrefixType = CrossDataSet[LeftIn, RightIn] })(fun: c.Expr[(LeftIn, RightIn) => Boolean]): c.Expr[DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[(LeftIn, RightIn)]
-
-    val stub = reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[(LeftIn, RightIn)] = c.Expr[UDT[(LeftIn, RightIn)]](createUdtOut).splice
-      new CrossFunctionBase[LeftIn, RightIn, (LeftIn, RightIn)] {
-        override def cross(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = {
-          val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-          val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-          if (fun.splice.apply(left, right)) {
-            val output = (left, right)
-            leftRecord.setNumFields(outputLength)
-            serializer.serialize(output, leftRecord)
-            out.collect(leftRecord)
-          }
-        }
-      }
-    }
-    val contract = reify {
-      val helper: CrossDataSet[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val builder = CrossOperator.builder(generatedStub).input1(leftInput).input2(rightInput)
-      
-      val ret = new CrossOperator(builder) with TwoInputScalaOperator[LeftIn, RightIn, (LeftIn, RightIn)] {
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-
-      }
-      new DataSet[(LeftIn, RightIn)](ret) with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)] {}
-    }
-
-    val result = c.Expr[DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSinkMacros.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSinkMacros.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSinkMacros.scala
deleted file mode 100644
index 0be49b4..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSinkMacros.scala
+++ /dev/null
@@ -1,346 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import java.io.DataOutput
-import java.io.OutputStream
-
-
-import org.apache.flink.api.scala.ScalaOutputFormat
-import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT, InputField}
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.codegen.UDTDescriptors
-import org.apache.flink.api.scala.codegen.MacroContextHolder
-
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.types.Record
-import org.apache.flink.api.common.io.{BinaryOutputFormat => JavaBinaryOutputFormat}
-import org.apache.flink.api.common.io.{SerializedOutputFormat => JavaSerializedOutputFormat}
-import org.apache.flink.api.java.record.io.{DelimitedOutputFormat => JavaDelimitedOutputFormat}
-import org.apache.flink.api.java.record.io.{CsvOutputFormat => JavaCsvOutputFormat}
-import org.apache.flink.api.java.record.io.{FileOutputFormat => JavaFileOutputFormat}
-import org.apache.flink.api.common.io.{OutputFormat => JavaOutputFormat}
-
-
-trait ScalaOutputFormatBase[In] extends ScalaOutputFormat[In] { this: JavaOutputFormat[_] =>
-  protected val udt: UDT[In]
-  lazy val udf: UDF1[In, Nothing] = new UDF1[In, Nothing](udt, UDT.NothingUDT)
-  def getUDF: UDF1[In, Nothing] = udf
-  protected var deserializer: UDTSerializer[In] = _
-
-  abstract override def configure(config: Configuration) {
-    super.configure(config)
-    this.deserializer = udf.getInputDeserializer
-  }
-}
-
-
-object RawOutputFormat {
-  def apply[In](writeFunction: (In, OutputStream) => Unit): ScalaOutputFormat[In] = macro impl[In]
-  
-  def impl[In: c.WeakTypeTag](c: Context)(writeFunction: c.Expr[(In, OutputStream) => Unit]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    
-    val pact4sFormat = reify {
-      
-      new JavaFileOutputFormat with ScalaOutputFormatBase[In] {
-        override val udt = c.Expr(createUdtIn).splice
-
-        override def writeRecord(record: Record) = {
-          val input = deserializer.deserializeRecyclingOn(record)
-          writeFunction.splice.apply(input, this.stream)
-        }
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaOutputFormat[In]](Block(List(udtIn), pact4sFormat.tree))
-
-    return result
-    
-  }
-}
-
-object BinaryOutputFormat {
-  
-  def apply[In](writeFunction: (In, DataOutput) => Unit): ScalaOutputFormat[In] = macro implWithoutBlocksize[In]
-  def apply[In](writeFunction: (In, DataOutput) => Unit, blockSize: Long): ScalaOutputFormat[In] = macro implWithBlocksize[In]
-  
-  def implWithoutBlocksize[In: c.WeakTypeTag](c: Context)(writeFunction: c.Expr[(In, DataOutput) => Unit]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(writeFunction, reify { None })
-  }
-  def implWithBlocksize[In: c.WeakTypeTag](c: Context)(writeFunction: c.Expr[(In, DataOutput) => Unit], blockSize: c.Expr[Long]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(writeFunction, reify { Some(blockSize.splice) })
-  }
-  
-  def impl[In: c.WeakTypeTag](c: Context)(writeFunction: c.Expr[(In, DataOutput) => Unit], blockSize: c.Expr[Option[Long]]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    
-    val pact4sFormat = reify {
-      
-      new JavaBinaryOutputFormat with ScalaOutputFormatBase[In] {
-        override val udt = c.Expr(createUdtIn).splice
-      
-        override def persistConfiguration(config: Configuration) {
-          blockSize.splice map { config.setLong(JavaBinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY, _) }
-        }
-
-        override def serialize(record: Record, target: DataOutput) = {
-          val input = deserializer.deserializeRecyclingOn(record)
-          writeFunction.splice.apply(input, target)
-        }
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaOutputFormat[In]](Block(List(udtIn), pact4sFormat.tree))
-
-    return result
-    
-  }
-}
-
-object BinarySerializedOutputFormat {
-  
-  def apply[In](): ScalaOutputFormat[In] = macro implWithoutBlocksize[In]
-  def apply[In](blockSize: Long): ScalaOutputFormat[In] = macro implWithBlocksize[In]
-  
-  def implWithoutBlocksize[In: c.WeakTypeTag](c: Context)() : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(reify { None })
-  }
-  def implWithBlocksize[In: c.WeakTypeTag](c: Context)(blockSize: c.Expr[Long]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(reify { Some(blockSize.splice) })
-  }
-  
-  def impl[In: c.WeakTypeTag](c: Context)(blockSize: c.Expr[Option[Long]]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    
-    val pact4sFormat = reify {
-      
-      new JavaSerializedOutputFormat with ScalaOutputFormat[In] {
-        override def persistConfiguration(config: Configuration) {
-          blockSize.splice map { config.setLong(JavaBinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY, _) }
-        }
-        val udt = c.Expr[UDT[In]](createUdtIn).splice
-        lazy val udf: UDF1[In, Nothing] = new UDF1[In, Nothing](udt, UDT.NothingUDT)
-        override def getUDF = udf
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaOutputFormat[In]](Block(List(udtIn), pact4sFormat.tree))
-
-    return result
-    
-  }
-}
-
-object DelimitedOutputFormat {
-  
-  def forString[In](formatFunction: In => String) = {
-
-    (source: In, target: Array[Byte]) => {
-      val str = formatFunction(source)
-      val data = str.getBytes
-      if (data.length <= target.length) {
-        System.arraycopy(data, 0, target, 0, data.length);
-        data.length;
-      } else {
-        -data.length;
-      }
-    }
-    
-  }
-
-  def forStringBuilder[In](formatFunction: (In, StringBuilder) => Unit)  = {
-
-    val stringBuilder = new StringBuilder
-
-    (source: In, target: Array[Byte]) => {
-      stringBuilder.clear
-      formatFunction(source, stringBuilder)
-
-      val data = stringBuilder.toString.getBytes
-      if (data.length <= target.length) {
-        System.arraycopy(data, 0, target, 0, data.length);
-        data.length;
-      } else {
-        -data.length;
-      }
-    }
-    
-  }
-
-  def maybeDelim(delim: String) = if (delim == null) None else Some(delim)
-  
-  def apply[In](formatFunction: In => String): ScalaOutputFormat[In] = macro writeFunctionForStringWithoutDelim[In]
-  def apply[In](formatFunction: In => String, delimiter: String): ScalaOutputFormat[In] = macro writeFunctionForStringWithDelim[In]
-  def apply[In](formatFunction: (In, StringBuilder) => Unit): ScalaOutputFormat[In] = macro writeFunctionForStringBuilderWithoutDelim[In]
-  def apply[In](formatFunction: (In, StringBuilder) => Unit, delimiter: String): ScalaOutputFormat[In] = macro writeFunctionForStringBuilderWithDelim[In]
-  
-  def writeFunctionForStringWithoutDelim[In: c.WeakTypeTag](c: Context)(formatFunction: c.Expr[In => String]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    val writeFun = reify {
-      forString[In](formatFunction.splice)
-    }
-    impl(c)(writeFun, reify { None })
-  }
-  
-  def writeFunctionForStringWithDelim[In: c.WeakTypeTag](c: Context)(formatFunction: c.Expr[In => String], delimiter: c.Expr[String]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    val writeFun = reify {
-      forString[In](formatFunction.splice)
-    }
-    impl(c)(writeFun, reify { Some(delimiter.splice) })
-  }
-  
-  def writeFunctionForStringBuilderWithoutDelim[In: c.WeakTypeTag](c: Context)(formatFunction: c.Expr[(In, StringBuilder) => Unit]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    val writeFun = reify {
-      forStringBuilder[In](formatFunction.splice)
-    }
-    impl(c)(writeFun, reify { None })
-  }
-  
-  def writeFunctionForStringBuilderWithDelim[In: c.WeakTypeTag](c: Context)(formatFunction: c.Expr[(In, StringBuilder) => Unit], delimiter: c.Expr[String]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    val writeFun = reify {
-      forStringBuilder[In](formatFunction.splice)
-    }
-    impl(c)(writeFun, reify { Some(delimiter.splice) })
-  }
-  
-  def impl[In: c.WeakTypeTag](c: Context)(writeFunction: c.Expr[(In, Array[Byte]) => Int], delimiter: c.Expr[Option[String]]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    
-    val pact4sFormat = reify {
-      
-      new JavaDelimitedOutputFormat with ScalaOutputFormatBase[In] {
-        override val udt = c.Expr(createUdtIn).splice
-
-        override def persistConfiguration(config: Configuration) {
-          delimiter.splice map { config.setString(JavaDelimitedOutputFormat.RECORD_DELIMITER, _) }
-        }
-
-        override def serializeRecord(record: Record, target: Array[Byte]): Int = {
-          val input = deserializer.deserializeRecyclingOn(record)
-          writeFunction.splice.apply(input, target)
-        }
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaOutputFormat[In]](Block(List(udtIn), pact4sFormat.tree))
-
-    return result
-    
-  }
-}
-
-object CsvOutputFormat {
-  def apply[In](recordDelimiter: Option[String], fieldDelimiter: Option[String] = None, lenient: Option[Boolean]): ScalaOutputFormat[In] = macro impl[In]
-  
-  def apply[In](): ScalaOutputFormat[In] = macro implWithoutAll[In]
-  def apply[In](recordDelimiter: String): ScalaOutputFormat[In] = macro implWithRD[In]
-  def apply[In](recordDelimiter: String, fieldDelimiter: String): ScalaOutputFormat[In] = macro implWithRDandFD[In]
-  def apply[In](recordDelimiter: String, fieldDelimiter: String, lenient: Boolean): ScalaOutputFormat[In] = macro implWithRDandFDandLenient[In]
-  
-  def implWithoutAll[In: c.WeakTypeTag](c: Context)() : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(reify { None }, reify { None }, reify { None })
-  }
-  def implWithRD[In: c.WeakTypeTag](c: Context)(recordDelimiter: c.Expr[String]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(reify { Some(recordDelimiter.splice) }, reify { None }, reify { None })
-  }
-  def implWithRDandFD[In: c.WeakTypeTag](c: Context)(recordDelimiter: c.Expr[String], fieldDelimiter: c.Expr[String]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(reify { Some(recordDelimiter.splice) }, reify { Some(fieldDelimiter.splice) }, reify { None })
-  }
-  def implWithRDandFDandLenient[In: c.WeakTypeTag](c: Context)(recordDelimiter: c.Expr[String], fieldDelimiter: c.Expr[String], lenient: c.Expr[Boolean]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    impl(c)(reify { Some(recordDelimiter.splice) }, reify { Some(fieldDelimiter.splice) }, reify { Some(lenient.splice) })
-  }
-  
-  def impl[In: c.WeakTypeTag](c: Context)(recordDelimiter: c.Expr[Option[String]], fieldDelimiter: c.Expr[Option[String]], lenient: c.Expr[Option[Boolean]]) : c.Expr[ScalaOutputFormat[In]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    
-    val pact4sFormat = reify {
-      
-      new JavaCsvOutputFormat with ScalaOutputFormat[In] {
-        override def persistConfiguration(config: Configuration) {
-
-          val fields = getUDF.inputFields.filter(_.isUsed)
-
-          config.setInteger(JavaCsvOutputFormat.NUM_FIELDS_PARAMETER, fields.length)
-
-          var index = 0
-          fields foreach { field: InputField =>
-            val tpe = getUDF.inputUDT.fieldTypes(field.localPos)
-            config.setClass(JavaCsvOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + index, tpe)
-            config.setInteger(JavaCsvOutputFormat.RECORD_POSITION_PARAMETER_PREFIX + index, field.localPos)
-            index = index + 1
-          }
-
-          recordDelimiter.splice map { config.setString(JavaCsvOutputFormat.RECORD_DELIMITER_PARAMETER, _) }
-          fieldDelimiter.splice map { config.setString(JavaCsvOutputFormat.FIELD_DELIMITER_PARAMETER, _) }
-          lenient.splice map { config.setBoolean(JavaCsvOutputFormat.LENIENT_PARSING, _) }
-        }
-        
-        val udt = c.Expr[UDT[In]](createUdtIn).splice
-        lazy val udf: UDF1[In, Nothing] = new UDF1[In, Nothing](udt, UDT.NothingUDT)
-        override def getUDF = udf
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaOutputFormat[In]](Block(List(udtIn), pact4sFormat.tree))
-
-    return result
-    
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSourceMacros.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSourceMacros.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSourceMacros.scala
deleted file mode 100644
index 643335d..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/DataSourceMacros.scala
+++ /dev/null
@@ -1,414 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import scala.language.experimental.macros
-import scala.reflect.macros.Context
-
-import java.io.DataInput
-
-import org.apache.flink.api.scala.ScalaInputFormat
-import org.apache.flink.api.scala.analysis.{UDTSerializer, OutputField, UDT}
-import org.apache.flink.api.scala.analysis.UDF0
-import org.apache.flink.api.scala.analysis.OutputField
-import org.apache.flink.api.scala.codegen.MacroContextHolder
-
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.types.Record
-import org.apache.flink.types.Value
-import org.apache.flink.types.DoubleValue
-import org.apache.flink.types.IntValue
-import org.apache.flink.types.LongValue
-import org.apache.flink.types.StringValue
-import org.apache.flink.types.parser.DoubleValueParser
-import org.apache.flink.types.parser.IntValueParser
-import org.apache.flink.types.parser.LongValueParser
-import org.apache.flink.types.parser.FieldParser
-import org.apache.flink.types.parser.StringValueParser
-import org.apache.flink.api.common.io.{InputFormat => JavaInputFormat}
-import org.apache.flink.api.common.io.{BinaryInputFormat => JavaBinaryInputFormat}
-import org.apache.flink.api.common.io.{SerializedInputFormat => JavaSerializedInputFormat}
-import org.apache.flink.api.java.record.io.{DelimitedInputFormat => JavaDelimitedInputFormat}
-import org.apache.flink.api.java.record.io.{FixedLengthInputFormat => JavaFixedLengthInputFormat}
-import org.apache.flink.api.java.record.io.{CsvInputFormat => JavaCsvInputFormat}
-import org.apache.flink.api.java.record.io.{TextInputFormat => JavaTextInputFormat}
-
-
-trait ScalaInputFormatBase[Out] extends ScalaInputFormat[Out] { this: JavaInputFormat[_, _] =>
-  protected val udt: UDT[Out]
-  
-  @transient private var udf: UDF0[Out] = _
-  
-  def getUDF: UDF0[Out] = {
-    if (udf == null) {
-      udf = new UDF0(udt)
-    }
-    udf
-  }
-  
-  @transient protected var serializer: UDTSerializer[Out] = _
-  @transient protected var outputLength: Int = _
-
-  abstract override def configure(config: Configuration) {
-    super.configure(config)
-    this.outputLength = getUDF.getOutputLength
-    this.serializer = getUDF.getOutputSerializer
-  }
-  
-  private def readObject(s: java.io.ObjectInputStream) = {
-    s.defaultReadObject();
-    this.udf = new UDF0(udt)
-  }
-}
-
-object BinaryInputFormat {
-  
-  // We need to do the "optional parameters" manually here (and in all other formats) because scala macros
-  // do (not yet?) support optional parameters in macros.
-  
-  def apply[Out](readFunction: DataInput => Out): ScalaInputFormat[Out] = macro implWithoutBlocksize[Out]
-  def apply[Out](readFunction: DataInput => Out, blockSize: Long): ScalaInputFormat[Out] = macro implWithBlocksize[Out]
-  
-  def implWithoutBlocksize[Out: c.WeakTypeTag](c: Context)(readFunction: c.Expr[DataInput => Out]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(readFunction, reify { None })
-  }
-  def implWithBlocksize[Out: c.WeakTypeTag](c: Context)(readFunction: c.Expr[DataInput => Out], blockSize: c.Expr[Long]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(readFunction, reify { Some(blockSize.splice) })
-  }
-  
-  def impl[Out: c.WeakTypeTag](c: Context)(readFunction: c.Expr[DataInput => Out], blockSize: c.Expr[Option[Long]]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-    
-    val pact4sFormat = reify {
-      
-      new JavaBinaryInputFormat[Record] with ScalaInputFormatBase[Out] {
-        override val udt = c.Expr(createUdtOut).splice
-
-        override def persistConfiguration(config: Configuration) {
-          super.persistConfiguration(config)
-          blockSize.splice map { config.setLong(JavaBinaryInputFormat.BLOCK_SIZE_PARAMETER_KEY, _) }
-        }
-
-        override def deserialize(record: Record, source: DataInput) = {
-          val output = readFunction.splice.apply(source)
-          record.setNumFields(outputLength)
-          serializer.serialize(output, record)
-          record
-        }
-      }
-    }
-    
-    val result = c.Expr[ScalaInputFormat[Out]](Block(List(udtOut), pact4sFormat.tree))
-
-//    c.info(c.enclosingPosition, s"GENERATED Pact4s DataSource Format: ${show(result)}", true)
-
-    return result
-    
-  }
-}
-
-object BinarySerializedInputFormat {
-  def apply[Out](): ScalaInputFormat[Out] = macro implWithoutBlocksize[Out]
-  def apply[Out](blockSize: Long): ScalaInputFormat[Out] = macro implWithBlocksize[Out]
-  
-  def implWithoutBlocksize[Out: c.WeakTypeTag](c: Context)() : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(reify { None })
-  }
-  def implWithBlocksize[Out: c.WeakTypeTag](c: Context)(blockSize: c.Expr[Long]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(reify { Some(blockSize.splice) })
-  }
-  
-  def impl[Out: c.WeakTypeTag](c: Context)(blockSize: c.Expr[Option[Long]]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-    
-    val pact4sFormat = reify {
-      
-      new JavaSerializedInputFormat[Record] with ScalaInputFormat[Out] {
-        override def persistConfiguration(config: Configuration) {
-          super.persistConfiguration(config)
-          blockSize.splice map { config.setLong(JavaBinaryInputFormat.BLOCK_SIZE_PARAMETER_KEY, _) }
-        }
-        
-        val udt: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-        val udf: UDF0[Out] = new UDF0(udt)
-        override def getUDF = udf
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaInputFormat[Out]](Block(List(udtOut), pact4sFormat.tree))
-
-//    c.info(c.enclosingPosition, s"GENERATED Pact4s DataSource Format: ${show(result)}", true)
-
-    return result
-    
-  }
-}
-
-object DelimitedInputFormat {
-  def asReadFunction[Out](parseFunction: String => Out) = {
-    (source: Array[Byte], offset: Int, numBytes: Int) => {
-        parseFunction(new String(source, offset, numBytes))
-    }
-  }
-  
-  def apply[Out](readFunction: (Array[Byte], Int, Int) => Out, delim: Option[String]): ScalaInputFormat[Out] = macro impl[Out]
-  def apply[Out](parseFunction: String => Out): ScalaInputFormat[Out] = macro parseFunctionImplWithoutDelim[Out]
-  def apply[Out](parseFunction: String => Out, delim: String): ScalaInputFormat[Out] = macro parseFunctionImplWithDelim[Out]
-  
-  def parseFunctionImplWithoutDelim[Out: c.WeakTypeTag](c: Context)(parseFunction: c.Expr[String => Out]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    val readFun = reify {
-      asReadFunction[Out](parseFunction.splice)
-    }
-    impl(c)(readFun, reify { None })
-  }
-  def parseFunctionImplWithDelim[Out: c.WeakTypeTag](c: Context)(parseFunction: c.Expr[String => Out], delim: c.Expr[String]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    val readFun = reify {
-      asReadFunction[Out](parseFunction.splice)
-    }
-    impl(c)(readFun, reify { Some(delim.splice) })
-  }
-
-  
-  def impl[Out: c.WeakTypeTag](c: Context)(readFunction: c.Expr[(Array[Byte], Int, Int) => Out], delim: c.Expr[Option[String]]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-    
-    val pact4sFormat = reify {
-      
-      new JavaDelimitedInputFormat with ScalaInputFormatBase[Out]{
-        override val udt = c.Expr(createUdtOut).splice
-        
-        setDelimiter((delim.splice.getOrElse("\n")));
-
-        override def readRecord(record: Record, source: Array[Byte], offset: Int, numBytes: Int): Record = {
-          val output = readFunction.splice.apply(source, offset, numBytes)
-
-          if (output != null) {
-            record.setNumFields(outputLength)
-            serializer.serialize(output, record)
-            record
-          } else {
-            null
-          }
-        }
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaInputFormat[Out]](Block(List(udtOut), pact4sFormat.tree))
-
-//    c.info(c.enclosingPosition, s"GENERATED Pact4s DataSource Format: ${show(result)}", true)
-
-    return result
-    
-  }
-}
-
-object CsvInputFormat {
-  
-  def apply[Out](): ScalaInputFormat[Out] = macro implWithoutAll[Out]
-  def apply[Out](recordDelim: String): ScalaInputFormat[Out] = macro implWithRD[Out]
-  def apply[Out](recordDelim: String, fieldDelim: Char): ScalaInputFormat[Out] = macro implWithRDandFD[Out]
-
-  def apply[Out](fieldIndices: Seq[Int]): ScalaInputFormat[Out] = macro implWithoutAllWithIndices[Out]
-  def apply[Out](fieldIndices: Seq[Int], recordDelim: String): ScalaInputFormat[Out] = macro implWithRDWithIndices[Out]
-  def apply[Out](fieldIndices: Seq[Int], recordDelim: String, fieldDelim: Char): ScalaInputFormat[Out] = macro implWithRDandFDWithIndices[Out]
-
-
-  def implWithoutAll[Out: c.WeakTypeTag](c: Context)() : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(reify { Seq[Int]() }, reify { None }, reify { None })
-  }
-  def implWithRD[Out: c.WeakTypeTag](c: Context)
-                                    (recordDelim: c.Expr[String]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(reify { Seq[Int]() },reify { Some(recordDelim.splice) }, reify { None })
-  }
-  def implWithRDandFD[Out: c.WeakTypeTag](c: Context)
-                                         (recordDelim: c.Expr[String], fieldDelim: c.Expr[Char]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(reify { Seq[Int]() },reify { Some(recordDelim.splice) }, reify { Some(fieldDelim.splice) })
-  }
-
-  def implWithoutAllWithIndices[Out: c.WeakTypeTag](c: Context)(fieldIndices: c.Expr[Seq[Int]]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(fieldIndices, reify { None }, reify { None })
-  }
-  def implWithRDWithIndices[Out: c.WeakTypeTag](c: Context)
-                                    (fieldIndices: c.Expr[Seq[Int]], recordDelim: c.Expr[String]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(fieldIndices,reify { Some(recordDelim.splice) }, reify { None })
-  }
-  def implWithRDandFDWithIndices[Out: c.WeakTypeTag](c: Context)
-                                         (fieldIndices: c.Expr[Seq[Int]], recordDelim: c.Expr[String], fieldDelim: c.Expr[Char]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    impl(c)(fieldIndices,reify { Some(recordDelim.splice) }, reify { Some(fieldDelim.splice) })
-  }
-  
-  def impl[Out: c.WeakTypeTag](c: Context)
-                              (fieldIndices: c.Expr[Seq[Int]], recordDelim: c.Expr[Option[String]], fieldDelim: c.Expr[Option[Char]]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-    
-    val pact4sFormat = reify {
-      new JavaCsvInputFormat with ScalaInputFormat[Out] {
-        
-        val udt: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-        val udf: UDF0[Out] = new UDF0(udt)
-        override def getUDF = udf
-        
-        setDelimiter((recordDelim.splice.getOrElse("\n")))
-        setFieldDelimiter(fieldDelim.splice.getOrElse(','))
-        
-        // there is a problem with the reification of Class[_ <: Value], so we work with Class[_] and convert it
-        // in a function outside the reify block
-//        setFieldTypesArray(asValueClassArray(getUDF.outputFields.filter(_.isUsed).map(x => getUDF.outputUDT.fieldTypes(x.localPos))))
-        
-        // is this maybe more correct? Note that null entries in the types array denote fields skipped by the CSV parser
-        val indices = fieldIndices.splice
-        val fieldTypes = asValueClassArrayFromOption(getUDF.outputFields.map {
-          case x if x.isUsed => Some(getUDF.outputUDT.fieldTypes(x.localPos))
-          case _ => None
-        })
-        if (indices.isEmpty) {
-          setFieldTypesArray(fieldTypes)
-        } else {
-          setFields(indices.toArray, fieldTypes)
-        }
-        
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaInputFormat[Out]](Block(List(udtOut), pact4sFormat.tree))
-
-//    c.info(c.enclosingPosition, s"GENERATED Pact4s DataSource Format: ${show(result)}", true)
-
-    return result
-    
-  }
-  
-    // we need to do this conversion outside the reify block
-    def asValueClassArray(types: Seq[Class[_]]) : Array[Class[_ <: Value]] = {
-      
-      val typed = types.foldRight(List[Class[_ <: Value]]())((x,y) => {
-        val t : Class[_ <: Value] = x.asInstanceOf[Class[_ <: Value]]
-        t :: y
-      })
-      
-      Array[Class[_ <: Value]]() ++ typed
-    }
-    
-    // we need to do this conversion outside the reify block
-    def asValueClassArrayFromOption(types: Seq[Option[Class[_]]]) : Array[Class[_ <: Value]] = {
-      
-      val typed = types.foldRight(List[Class[_ <: Value]]())((x,y) => {
-        val t : Class[_ <: Value] = x match {
-          case None => null
-          case Some(x) => x.asInstanceOf[Class[_ <: Value]]
-        }
-        t :: y
-      })
-      
-      Array[Class[_ <: Value]]() ++ typed
-    }
-}
-
-object TextInputFormat {
-  def apply(charSetName: Option[String] = None): ScalaInputFormat[String] = {
-
-    new JavaTextInputFormat with ScalaInputFormat[String] {
-      override def persistConfiguration(config: Configuration) {
-        super.persistConfiguration(config)
-
-        charSetName map { config.setString(JavaTextInputFormat.CHARSET_NAME, _) }
-
-        config.setInteger(JavaTextInputFormat.FIELD_POS, getUDF.outputFields(0).localPos)
-      }
-     // override val udt: UDT[String] = UDT.StringUDT
-      val udf: UDF0[String] = new UDF0(UDT.StringUDT)
-      override def getUDF = udf
-    }
-  }
-}
-
-object FixedLengthInputFormat {
-  def apply[Out](readFunction: (Array[Byte], Int) => Out, recordLength: Int): ScalaInputFormat[Out] = macro impl[Out]
-  
-  def impl[Out: c.WeakTypeTag](c: Context)(readFunction: c.Expr[(Array[Byte], Int) => Out], recordLength: c.Expr[Int]) : c.Expr[ScalaInputFormat[Out]] = {
-    import c.universe._
-    
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-    
-    val pact4sFormat = reify {
-      
-      new JavaFixedLengthInputFormat with ScalaInputFormatBase[Out] {
-        override val udt = c.Expr(createUdtOut).splice
-
-        override def persistConfiguration(config: Configuration) {
-          super.persistConfiguration(config)
-          config.setInteger(JavaFixedLengthInputFormat.RECORDLENGTH_PARAMETER_KEY, (recordLength.splice))
-        }
-
-        override def readBytes(record: Record, source: Array[Byte], startPos: Int): Boolean = {
-          val output = readFunction.splice.apply(source, startPos)
-
-          if (output != null) {
-            record.setNumFields(outputLength)
-            serializer.serialize(output, record)
-          }
-
-          return output != null
-        }
-      }
-      
-    }
-    
-    val result = c.Expr[ScalaInputFormat[Out]](Block(List(udtOut), pact4sFormat.tree))
-
-//    c.info(c.enclosingPosition, s"GENERATED Pact4s DataSource Format: ${show(result)}", true)
-
-    return result
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala
deleted file mode 100644
index 66d94ba..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/IterateOperators.scala
+++ /dev/null
@@ -1,235 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.codegen.MacroContextHolder
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis.UDT
-import org.apache.flink.api.scala.analysis.UDT.NothingUDT
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.analysis.UDTSerializer
-import org.apache.flink.api.scala.analysis.UDF0
-import org.apache.flink.api.scala.analysis.FieldSelector
-
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.api.common.functions.AbstractRichFunction
-import org.apache.flink.api.java.record.operators.BulkIteration
-import org.apache.flink.api.common.operators.base.BulkIterationBase
-import org.apache.flink.api.java.record.operators.DeltaIteration
-import org.apache.flink.api.common.operators.base.BulkIterationBase.{TerminationCriterionAggregationConvergence, TerminationCriterionAggregator, TerminationCriterionMapper}
-import org.apache.flink.api.common.operators.base.MapOperatorBase
-import org.apache.flink.types.NothingTypeInfo
-import org.apache.flink.types.{Nothing => JavaNothing}
-import org.apache.flink.api.java.typeutils.RecordTypeInfo
-import org.apache.flink.api.common.operators.{UnaryOperatorInformation, Operator}
-import org.apache.flink.api.java.record.operators.MapOperator
-import org.apache.flink.types.Record
-import org.apache.flink.api.java.record.functions.MapFunction
-import org.apache.flink.util.Collector
-
-object IterateMacros {
-
-  def iterateWithDelta[SolutionItem: c.WeakTypeTag, DeltaItem: c.WeakTypeTag](c: Context { type PrefixType = DataSet[SolutionItem] })(stepFunction: c.Expr[DataSet[SolutionItem] => (DataSet[SolutionItem], DataSet[DeltaItem])]): c.Expr[DataSet[SolutionItem]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtSolution, createUdtSolution) = slave.mkUdtClass[SolutionItem]
-    val (udtDelta, createUdtDelta) = slave.mkUdtClass[DeltaItem]
-
-    val contract = reify {
-      val solutionUDT = c.Expr[UDT[SolutionItem]](createUdtSolution).splice
-      val contract = new BulkIteration with BulkIterationScalaOperator[SolutionItem] {
-        val udf = new UDF0[SolutionItem](solutionUDT)
-        override def getUDF = udf
-        private val inputPlaceHolder2 = new BulkIteration.PartialSolutionPlaceHolder(this) with ScalaOperator[SolutionItem, Record] with Serializable {
-          val udf = new UDF0[SolutionItem](solutionUDT)
-          override def getUDF = udf
-          
-        }
-        override def getPartialSolution: Operator[Record] = inputPlaceHolder2.asInstanceOf[Operator[Record]]
-      }
-      
-      val partialSolution = new DataSet(contract.getPartialSolution().asInstanceOf[Operator[Record] with ScalaOperator[SolutionItem, Record]])
-
-      val (output, term) = stepFunction.splice.apply(partialSolution)
-
-      contract.setInput(c.prefix.splice.contract)
-      contract.setNextPartialSolution(output.contract)
-
-      // is currently not implemented in flink
-//      if (term != null) contract.setTerminationCriterion(term)
-
-      new DataSet(contract)
-    }
-
-    val result = c.Expr[DataSet[SolutionItem]](Block(List(udtSolution, udtDelta), contract.tree))
-
-    return result
-  }
-  
-  def iterate[SolutionItem: c.WeakTypeTag](c: Context { type PrefixType = DataSet[SolutionItem] })(n: c.Expr[Int], stepFunction: c.Expr[DataSet[SolutionItem] => DataSet[SolutionItem]]): c.Expr[DataSet[SolutionItem]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtSolution, createUdtSolution) = slave.mkUdtClass[SolutionItem]
-
-    val contract = reify {
-      val solutionUDT = c.Expr[UDT[SolutionItem]](createUdtSolution).splice
-      val contract = new BulkIteration with BulkIterationScalaOperator[SolutionItem] {
-        val udf = new UDF0[SolutionItem](solutionUDT)
-        override def getUDF = udf
-        private val inputPlaceHolder2 = new BulkIteration.PartialSolutionPlaceHolder(this) with ScalaOperator[SolutionItem, Record] with Serializable {
-          val udf = new UDF0[SolutionItem](solutionUDT)
-          override def getUDF = udf
-          
-        }
-        override def getPartialSolution: Operator[Record] = inputPlaceHolder2.asInstanceOf[Operator[Record]]
-      }
-      
-      val partialSolution = new DataSet(contract.getPartialSolution().asInstanceOf[Operator[Record] with ScalaOperator[SolutionItem, Record]])
-
-      val output = stepFunction.splice.apply(partialSolution)
-
-      contract.setInput(c.prefix.splice.contract)
-      contract.setNextPartialSolution(output.contract)
-      contract.setMaximumNumberOfIterations(n.splice)
-
-      new DataSet(contract)
-    }
-
-    val result = c.Expr[DataSet[SolutionItem]](Block(List(udtSolution), contract.tree))
-
-    return result
-  }
-
-  def iterateWithTermination[SolutionItem: c.WeakTypeTag, TerminationItem: c.WeakTypeTag](c: Context { type
-  PrefixType = DataSet[SolutionItem] })(n: c.Expr[Int], stepFunction: c.Expr[DataSet[SolutionItem] => 
-    DataSet[SolutionItem]], terminationFunction: c.Expr[(DataSet[SolutionItem], 
-    DataSet[SolutionItem]) => DataSet[TerminationItem]]): c.Expr[DataSet[SolutionItem]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-
-    val (udtSolution, createUdtSolution) = slave.mkUdtClass[SolutionItem]
-    val (udtTermination, createUdtTermination) = slave.mkUdtClass[TerminationItem]
-
-    val contract = reify {
-      val solutionUDT = c.Expr[UDT[SolutionItem]](createUdtSolution).splice
-      val terminationUDT = c.Expr[UDT[TerminationItem]](createUdtTermination).splice
-      val contract = new BulkIteration with BulkIterationScalaOperator[SolutionItem] {
-        val udf = new UDF0[SolutionItem](solutionUDT)
-        override def getUDF = udf
-        private val inputPlaceHolder2 = new BulkIteration.PartialSolutionPlaceHolder(this) with ScalaOperator[SolutionItem, Record] with Serializable {
-          val udf = new UDF0[SolutionItem](solutionUDT)
-          override def getUDF = udf
-
-        }
-        override def getPartialSolution: Operator[Record] = inputPlaceHolder2.asInstanceOf[Operator[Record]]
-      }
-
-      val partialSolution = new DataSet(contract.getPartialSolution().asInstanceOf[Operator[Record] with ScalaOperator[SolutionItem, Record]])
-
-      val output = stepFunction.splice.apply(partialSolution)
-      val terminationCriterion = terminationFunction.splice.apply(partialSolution, output)
-
-
-      contract.setInput(c.prefix.splice.contract)
-      contract.setNextPartialSolution(output.contract)
-      contract.setMaximumNumberOfIterations(n.splice)
-      contract.setTerminationCriterion(terminationCriterion.contract)
-
-      new DataSet(contract)
-    }
-
-    val result = c.Expr[DataSet[SolutionItem]](Block(List(udtSolution, udtTermination), contract.tree))
-
-    return result
-  }
-}
-
-
-object WorksetIterateMacros {
-
-   
-  def iterateWithDelta[SolutionItem: c.WeakTypeTag, SolutionKey: c.WeakTypeTag, WorksetItem: c.WeakTypeTag](c: Context { type PrefixType = DataSet[SolutionItem] })(workset: c.Expr[DataSet[WorksetItem]], solutionSetKey: c.Expr[SolutionItem => SolutionKey], stepFunction: c.Expr[(DataSet[SolutionItem], DataSet[WorksetItem]) => (DataSet[SolutionItem], DataSet[WorksetItem])], maxIterations: c.Expr[Int]): c.Expr[DataSet[SolutionItem]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtSolution, createUdtSolution) = slave.mkUdtClass[SolutionItem]
-    val (udtWorkset, createUdtWorkset) = slave.mkUdtClass[WorksetItem]
-
-    val keySelection = slave.getSelector(solutionSetKey)
-
-    val contract = reify {
-      
-      val solutionUDT = c.Expr[UDT[SolutionItem]](createUdtSolution).splice
-      val worksetUDT = c.Expr[UDT[WorksetItem]](createUdtWorkset).splice
-      
-      val keySelector = new FieldSelector(solutionUDT, keySelection.splice)
-      val keyFields = keySelector.selectedFields
-      val keyPositions = keyFields.toIndexArray
-
-      val contract = new DeltaIteration(keyPositions) with DeltaIterationScalaOperator[SolutionItem] {
-        override val key = keySelector
-        val udf = new UDF0[SolutionItem](solutionUDT)     
-        override def getUDF = udf
-
-        private val solutionSetPlaceHolder2 = new DeltaIteration.SolutionSetPlaceHolder(this) with ScalaOperator[SolutionItem, Record] with Serializable {
-          val udf = new UDF0[SolutionItem](solutionUDT)
-          override def getUDF = udf
-
-        }
-        override def getSolutionSet: Operator[Record] = solutionSetPlaceHolder2.asInstanceOf[Operator[Record]]
-        
-        private val worksetPlaceHolder2 = new DeltaIteration.WorksetPlaceHolder(this) with ScalaOperator[WorksetItem, Record] with Serializable {
-          val udf = new UDF0[WorksetItem](worksetUDT)
-          override def getUDF = udf
-
-        }
-        override def getWorkset: Operator[Record] = worksetPlaceHolder2.asInstanceOf[Operator[Record]]
-      }
-
-      val solutionInput = new DataSet(contract.getSolutionSet().asInstanceOf[Operator[Record] with ScalaOperator[SolutionItem, Record]])
-      val worksetInput = new DataSet(contract.getWorkset().asInstanceOf[Operator[Record] with ScalaOperator[WorksetItem, Record]])
-
-
-      contract.setInitialSolutionSet(c.prefix.splice.contract)
-      contract.setInitialWorkset(workset.splice.contract)
-
-      val (delta, nextWorkset) = stepFunction.splice.apply(solutionInput, worksetInput)
-      contract.setSolutionSetDelta(delta.contract)
-      contract.setNextWorkset(nextWorkset.contract)
-      contract.setMaximumNumberOfIterations(maxIterations.splice)
-
-      new DataSet(contract)
-    }
-    
-    val result = c.Expr[DataSet[SolutionItem]](Block(List(udtSolution, udtWorkset), contract.tree))
-
-    return result
-  }
-}


[43/60] git commit: [scala] Add Scalastyle, use scalastyle-config.xml from Spark

Posted by al...@apache.org.
[scala] Add Scalastyle, use scalastyle-config.xml from Spark


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

Branch: refs/heads/master
Commit: 0385651f809b68e2f260cfee9c26511df3788341
Parents: fd28098
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 16 12:34:08 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../scala/graph/ConnectedComponents.scala       |  11 +-
 .../scala/graph/EnumTrianglesBasic.scala        | 214 ++++++++++---------
 .../examples/scala/graph/PageRankBasic.scala    |   4 +-
 .../scala/graph/TransitiveClosureNaive.scala    | 177 +++++++--------
 .../examples/scala/misc/PiEstimation.scala      |   2 +-
 .../examples/scala/ml/LinearRegression.scala    | 204 +++++++++---------
 .../org/apache/flink/api/scala/DataSet.scala    |  35 ++-
 .../flink/api/scala/ExecutionEnvironment.scala  |  31 ++-
 .../apache/flink/api/scala/GroupedDataSet.scala |   2 +-
 .../apache/flink/api/scala/coGroupDataSet.scala |   4 +-
 .../flink/api/scala/codegen/Counter.scala       |   6 +-
 .../api/scala/codegen/MacroContextHolder.scala  |   8 +-
 .../flink/api/scala/codegen/TreeGen.scala       |   9 +-
 .../flink/api/scala/codegen/TypeAnalyzer.scala  |  21 +-
 .../api/scala/codegen/TypeDescriptors.scala     |  17 +-
 .../api/scala/codegen/TypeInformationGen.scala  |  21 +-
 .../apache/flink/api/scala/crossDataSet.scala   |   2 +-
 .../apache/flink/api/scala/joinDataSet.scala    |   2 +-
 .../org/apache/flink/api/scala/package.scala    |   4 +-
 .../scala/typeutils/ScalaTupleComparator.scala  |   3 +-
 .../scala/typeutils/ScalaTupleSerializer.scala  |   3 +-
 .../scala/typeutils/ScalaTupleTypeInfo.scala    |   5 +-
 .../flink/api/scala/typeutils/TypeUtils.scala   |   3 +-
 .../api/scala/unfinishedKeyPairOperation.scala  |   4 +-
 .../api/scala/ScalaAPICompletenessTest.scala    |  21 +-
 .../SemanticPropertiesTranslationTest.scala     |   8 +-
 .../DeltaIterationTranslationTest.scala         |  10 +-
 .../translation/ReduceTranslationTest.scala     |   4 +-
 .../api/scala/runtime/TupleSerializerTest.scala |   4 +-
 pom.xml                                         |  27 ++-
 tools/maven/scalastyle-config.xml               | 146 +++++++++++++
 32 files changed, 627 insertions(+), 386 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index af23bc2..1bca0cf 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,4 +1,5 @@
 .cache
+scalastyle-output.xml
 .classpath
 .idea
 .metadata

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
index df28ad0..e00b2dd 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
@@ -71,7 +71,8 @@ object ConnectedComponents {
     // assign the initial components (equal to the vertex id)
     val vertices = getVerticesDataSet(env).map { id => (id, id) }
 
-    // undirected edges by emitting for each input edge the input edges itself and an inverted version
+    // undirected edges by emitting for each input edge the input edges itself and an inverted
+    // version
     val edges = getEdgesDataSet(env).flatMap { edge => Seq(edge, (edge._2, edge._1)) }
 
     // open a delta iteration
@@ -106,20 +107,22 @@ object ConnectedComponents {
  
   private def parseParameters(args: Array[String]): Boolean = {
     if (args.length > 0) {
-        fileOutput = true
+      fileOutput = true
       if (args.length == 4) {
         verticesPath = args(0)
         edgesPath = args(1)
         outputPath = args(2)
         maxIterations = args(3).toInt
       } else {
-        System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>")
+        System.err.println("Usage: ConnectedComponents <vertices path> <edges path> <result path>" +
+          " <max number of iterations>")
         false
       }
     } else {
       System.out.println("Executing Connected Components example with built-in default data.")
       System.out.println("  Provide parameters to read input data from a file.")
-      System.out.println("  Usage: ConnectedComponents <vertices path> <edges path> <result path> <max number of iterations>")
+      System.out.println("  Usage: ConnectedComponents <vertices path> <edges path> <result path>" +
+        " <max number of iterations>")
     }
     true
   }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
index fe121d5..da1c078 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
@@ -65,111 +65,113 @@ import scala.collection.mutable
  *
  */
 object EnumTrianglesBasic {
-	
-	def main(args: Array[String]) {
-		if (!parseParameters(args)) {
-			return
-		}
-
-		// set up execution environment
-		val env = ExecutionEnvironment.getExecutionEnvironment
-
-		// read input data
-		val edges = getEdgeDataSet(env)
-		
-		// project edges by vertex id
-		val edgesById = edges map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) )
-		
-		val triangles = edgesById
-						// build triads
-						.groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
-						// filter triads
-						.join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => t }
-		
-		// emit result
-		if (fileOutput) {
-			triangles.writeAsCsv(outputPath, "\n", ",")
-		} else {
-			triangles.print()
-		}
-		
-		// execute program
-		env.execute("TriangleEnumeration Example")
-	}
-
-	// *************************************************************************
-	//     USER DATA TYPES
-	// *************************************************************************
-
-	case class Edge(v1: Int, v2: Int) extends Serializable
-	case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
-	
-		
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 *  Builds triads (triples of vertices) from pairs of edges that share a vertex.
-	 *  The first vertex of a triad is the shared vertex, the second and third vertex are ordered by vertexId. 
-	 *  Assumes that input edges share the first vertex and are in ascending order of the second vertex.
-	 */
-	class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
-
-		val vertices = mutable.MutableList[Integer]()
-		
-		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
-			
-			// clear vertex list
-			vertices.clear()
-
-			// build and emit triads
-			for(e <- edges.asScala) {
-			
-				// combine vertex with all previously read vertices
-				for(v <- vertices) {
-					out.collect(Triad(e.v1, v, e.v2))
-				}
-				vertices += e.v2
-			}
-		}
-	}
-
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-
-	private def parseParameters(args: Array[String]): Boolean = {
-		if (args.length > 0) {
-			fileOutput = true
-			if (args.length == 2) {
-				edgePath = args(0)
-				outputPath = args(1)
-			} else {
-				System.err.println("Usage: EnumTriangleBasic <edge path> <result path>")
-				false
-			}
-		} else {
-			System.out.println("Executing Enum Triangles Basic example with built-in default data.")
-			System.out.println("  Provide parameters to read input data from files.")
-			System.out.println("  See the documentation for the correct format of input files.")
-			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>")
-		}
-		true
-	}
-
-	private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
-		if (fileOutput) {
-			env.readCsvFile[Edge](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1))
-		} else {
-			val edges = EnumTrianglesData.EDGES.map{ case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int]) }
-			env.fromCollection(edges)
-		}
-	}
-	
-	
-	private var fileOutput: Boolean = false
-	private var edgePath: String = null
-	private var outputPath: String = null
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    // set up execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    // read input data
+    val edges = getEdgeDataSet(env)
+    
+    // project edges by vertex id
+    val edgesById = edges map(e => if (e.v1 < e.v2) e else Edge(e.v2, e.v1) )
+    
+    val triangles = edgesById
+            // build triads
+            .groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
+            // filter triads
+            .join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => t }
+    
+    // emit result
+    if (fileOutput) {
+      triangles.writeAsCsv(outputPath, "\n", ",")
+    } else {
+      triangles.print()
+    }
+    
+    // execute program
+    env.execute("TriangleEnumeration Example")
+  }
+
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+
+  case class Edge(v1: Int, v2: Int) extends Serializable
+  case class Triad(v1: Int, v2: Int, v3: Int) extends Serializable
+  
+    
+  // *************************************************************************
+  //     USER FUNCTIONS
+  // *************************************************************************
+
+  /**
+   *  Builds triads (triples of vertices) from pairs of edges that share a vertex. The first vertex
+   *  of a triad is the shared vertex, the second and third vertex are ordered by vertexId. Assumes
+   *  that input edges share the first vertex and are in ascending order of the second vertex.
+   */
+  class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
+
+    val vertices = mutable.MutableList[Integer]()
+    
+    override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
+      
+      // clear vertex list
+      vertices.clear()
+
+      // build and emit triads
+      for(e <- edges.asScala) {
+      
+        // combine vertex with all previously read vertices
+        for(v <- vertices) {
+          out.collect(Triad(e.v1, v, e.v2))
+        }
+        vertices += e.v2
+      }
+    }
+  }
+
+  // *************************************************************************
+  //     UTIL METHODS
+  // *************************************************************************
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      fileOutput = true
+      if (args.length == 2) {
+        edgePath = args(0)
+        outputPath = args(1)
+      } else {
+        System.err.println("Usage: EnumTriangleBasic <edge path> <result path>")
+        false
+      }
+    } else {
+      System.out.println("Executing Enum Triangles Basic example with built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>")
+    }
+    true
+  }
+
+  private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge] = {
+    if (fileOutput) {
+      env.readCsvFile[Edge](edgePath, fieldDelimiter = ' ', includedFields = Array(0, 1))
+    } else {
+      val edges = EnumTrianglesData.EDGES.map {
+        case Array(v1, v2) => new Edge(v1.asInstanceOf[Int], v2.asInstanceOf[Int])
+      }
+      env.fromCollection(edges)
+    }
+  }
+  
+  
+  private var fileOutput: Boolean = false
+  private var edgePath: String = null
+  private var outputPath: String = null
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
index 5f515c4..7fc9af7 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
@@ -86,7 +86,9 @@ object PageRankBasic {
       // initialize lists
       .map(e => AdjacencyList(e.sourceId, Array(e.targetId)))
       // concatenate lists
-      .groupBy("sourceId").reduce((l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds))
+      .groupBy("sourceId").reduce {
+      (l1, l2) => AdjacencyList(l1.sourceId, l1.targetIds ++ l2.targetIds)
+      }
 
     // start iteration
     val finalRanks = pagesWithRanks.iterateWithTermination(maxIterations) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
index 5416bb4..68f57e4 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
@@ -23,90 +23,93 @@ import org.apache.flink.util.Collector
 
 object TransitiveClosureNaive {
 
-	def main (args: Array[String]): Unit = {
-		if (!parseParameters(args)) {
-			return
-		}
-
-		val env = ExecutionEnvironment.getExecutionEnvironment
-
-		val edges = getEdgesDataSet(env)
-
-		val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long, Long)] =>
-
-			val nextPaths = prevPaths
-				.join(edges)
-				.where(1).equalTo(0) {
-					(left, right) => (left._1,right._2)
-				}
-				.union(prevPaths)
-				.groupBy(0, 1)
-				.reduce((l, r) => l)
-
-			val terminate = prevPaths
-				.coGroup(nextPaths)
-				.where(0).equalTo(0) {
-					(prev, next, out: Collector[(Long, Long)]) => {
-						val prevPaths = prev.toSet
-						for (n <- next)
-							if (!prevPaths.contains(n))
-								out.collect(n)
-					}
-			}
-			(nextPaths, terminate)
-		}
-
-		if (fileOutput)
-			paths.writeAsCsv(outputPath, "\n", " ")
-		else
-			paths.print()
-
-		env.execute("Scala Transitive Closure Example")
-
-
-	}
-
-
-	private var fileOutput: Boolean = false
-	private var edgesPath: String = null
-	private var outputPath: String = null
-	private var maxIterations: Int = 10
-
-	private def parseParameters(programArguments: Array[String]): Boolean = {
-		if (programArguments.length > 0) {
-			fileOutput = true
-			if (programArguments.length == 3) {
-				edgesPath = programArguments(0)
-				outputPath = programArguments(1)
-				maxIterations = Integer.parseInt(programArguments(2))
-			}
-			else {
-				System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of iterations>")
-				return false
-			}
-		}
-		else {
-			System.out.println("Executing TransitiveClosure example with default parameters and built-in default data.")
-			System.out.println("  Provide parameters to read input data from files.")
-			System.out.println("  See the documentation for the correct format of input files.")
-			System.out.println("  Usage: TransitiveClosure <edges path> <result path> <max number of iterations>")
-		}
-		true
-	}
-
-	private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = {
-		if (fileOutput) {
-			env.readCsvFile[(Long, Long)](
-				edgesPath,
-				fieldDelimiter = ' ',
-				includedFields = Array(0, 1))
-				.map { x => (x._1, x._2)}
-		}
-		else {
-			val edgeData = ConnectedComponentsData.EDGES map {
-				case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
-			}
-			env.fromCollection(edgeData)
-		}
-	}
-}
\ No newline at end of file
+  def main (args: Array[String]): Unit = {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    val edges = getEdgesDataSet(env)
+
+    val paths = edges.iterateWithTermination(maxIterations) { prevPaths: DataSet[(Long, Long)] =>
+
+      val nextPaths = prevPaths
+        .join(edges)
+        .where(1).equalTo(0) {
+          (left, right) => (left._1,right._2)
+        }
+        .union(prevPaths)
+        .groupBy(0, 1)
+        .reduce((l, r) => l)
+
+      val terminate = prevPaths
+        .coGroup(nextPaths)
+        .where(0).equalTo(0) {
+          (prev, next, out: Collector[(Long, Long)]) => {
+            val prevPaths = prev.toSet
+            for (n <- next)
+              if (!prevPaths.contains(n)) out.collect(n)
+          }
+      }
+      (nextPaths, terminate)
+    }
+
+    if (fileOutput) {
+      paths.writeAsCsv(outputPath, "\n", " ")
+    } else {
+      paths.print()
+    }
+
+    env.execute("Scala Transitive Closure Example")
+
+
+  }
+
+
+  private var fileOutput: Boolean = false
+  private var edgesPath: String = null
+  private var outputPath: String = null
+  private var maxIterations: Int = 10
+
+  private def parseParameters(programArguments: Array[String]): Boolean = {
+    if (programArguments.length > 0) {
+      fileOutput = true
+      if (programArguments.length == 3) {
+        edgesPath = programArguments(0)
+        outputPath = programArguments(1)
+        maxIterations = Integer.parseInt(programArguments(2))
+      }
+      else {
+        System.err.println("Usage: TransitiveClosure <edges path> <result path> <max number of " +
+          "iterations>")
+        return false
+      }
+    }
+    else {
+      System.out.println("Executing TransitiveClosure example with default parameters and " +
+        "built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  Usage: TransitiveClosure <edges path> <result path> <max number of " +
+        "iterations>")
+    }
+    true
+  }
+
+  private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[(Long, Long)] = {
+    if (fileOutput) {
+      env.readCsvFile[(Long, Long)](
+        edgesPath,
+        fieldDelimiter = ' ',
+        includedFields = Array(0, 1))
+        .map { x => (x._1, x._2)}
+    }
+    else {
+      val edgeData = ConnectedComponentsData.EDGES map {
+        case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
+      }
+      env.fromCollection(edgeData)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
index bb66b10..d08e6e2 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/misc/PiEstimation.scala
@@ -37,7 +37,7 @@ object PiEstimation {
           val y = Math.random()
           if (x * x + y * y < 1) 1L else 0L
         }
-        .reduce(_+_)
+        .reduce(_ + _)
 
     // ratio of samples in upper right quadrant vs total samples gives surface of upper
     // right quadrant, times 4 gives surface of whole unit circle, i.e. PI

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
index 95dcb9a..eed11a5 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
@@ -62,101 +62,101 @@ import scala.collection.JavaConverters._
  */
 object LinearRegression {
 
-	def main(args: Array[String]) {
-		if (!parseParameters(args)) {
-			return
-		}
-
-		val env = ExecutionEnvironment.getExecutionEnvironment
-		val data = getDataSet(env)
-		val parameters = getParamsDataSet(env)
-
-		val result = parameters.iterate(numIterations) { currentParameters =>
-			val newParameters = data
-				.map(new SubUpdate).withBroadcastSet(currentParameters, "parameters")
-				.reduce { (p1, p2) =>
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val data = getDataSet(env)
+    val parameters = getParamsDataSet(env)
+
+    val result = parameters.iterate(numIterations) { currentParameters =>
+      val newParameters = data
+        .map(new SubUpdate).withBroadcastSet(currentParameters, "parameters")
+        .reduce { (p1, p2) =>
           val result = p1._1 + p2._1
-				  (result, p1._2 + p2._2)
-			  }
-				.map { x => x._1.div(x._2) }
-			newParameters
-		}
-
-		if (fileOutput) {
-			result.writeAsText(outputPath)
-		}
-		else {
-			result.print()
-		}
-		env.execute("Scala Linear Regression example")
-	}
-
-	/**
-	 * A simple data sample, x means the input, and y means the target.
-	 */
+          (result, p1._2 + p2._2)
+        }
+        .map { x => x._1.div(x._2) }
+      newParameters
+    }
+
+    if (fileOutput) {
+      result.writeAsText(outputPath)
+    }
+    else {
+      result.print()
+    }
+    env.execute("Scala Linear Regression example")
+  }
+
+  /**
+   * A simple data sample, x means the input, and y means the target.
+   */
   case class Data(var x: Double, var y: Double)
 
-	/**
-	 * A set of parameters -- theta0, theta1.
-	 */
+  /**
+   * A set of parameters -- theta0, theta1.
+   */
   case class Params(theta0: Double, theta1: Double) {
     def div(a: Int): Params = {
       Params(theta0 / a, theta1 / a)
     }
 
-    def +(other: Params) = {
+    def + (other: Params) = {
       Params(theta0 + other.theta0, theta1 + other.theta1)
     }
   }
 
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
+  // *************************************************************************
+  //     USER FUNCTIONS
+  // *************************************************************************
 
-	/**
-	 * Compute a single BGD type update for every parameters.
-	 */
-	class SubUpdate extends RichMapFunction[Data, (Params, Int)] {
+  /**
+   * Compute a single BGD type update for every parameters.
+   */
+  class SubUpdate extends RichMapFunction[Data, (Params, Int)] {
 
-		private var parameter: Params = null
+    private var parameter: Params = null
 
-		/** Reads the parameters from a broadcast variable into a collection. */
-		override def open(parameters: Configuration) {
-			val parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala
+    /** Reads the parameters from a broadcast variable into a collection. */
+    override def open(parameters: Configuration) {
+      val parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala
       parameter = parameters.head
-		}
+    }
 
-		def map(in: Data): (Params, Int) = {
-			val theta0 =
+    def map(in: Data): (Params, Int) = {
+      val theta0 =
         parameter.theta0 - 0.01 * ((parameter.theta0 + (parameter.theta1 * in.x)) - in.y)
-			val theta1 =
+      val theta1 =
         parameter.theta1 - 0.01 * (((parameter.theta0 + (parameter.theta1 * in.x)) - in.y) * in.x)
-			(Params(theta0, theta1), 1)
-		}
-	}
-
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	private var fileOutput: Boolean = false
-	private var dataPath: String = null
-	private var outputPath: String = null
-	private var numIterations: Int = 10
-
-	private def parseParameters(programArguments: Array[String]): Boolean = {
-		if (programArguments.length > 0) {
-			fileOutput = true
-			if (programArguments.length == 3) {
-				dataPath = programArguments(0)
-				outputPath = programArguments(1)
-				numIterations = programArguments(2).toInt
-			}
-			else {
-				System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>")
-				false
-			}
-		}
-		else {
+      (Params(theta0, theta1), 1)
+    }
+  }
+
+  // *************************************************************************
+  //     UTIL METHODS
+  // *************************************************************************
+  private var fileOutput: Boolean = false
+  private var dataPath: String = null
+  private var outputPath: String = null
+  private var numIterations: Int = 10
+
+  private def parseParameters(programArguments: Array[String]): Boolean = {
+    if (programArguments.length > 0) {
+      fileOutput = true
+      if (programArguments.length == 3) {
+        dataPath = programArguments(0)
+        outputPath = programArguments(1)
+        numIterations = programArguments(2).toInt
+      }
+      else {
+        System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>")
+        false
+      }
+    }
+    else {
       System.out.println("Executing Linear Regression example with default parameters and " +
         "built-in default data.")
       System.out.println("  Provide parameters to read input data from files.")
@@ -164,30 +164,30 @@ object LinearRegression {
       System.out.println("  We provide a data generator to create synthetic input files for this " +
         "program.")
       System.out.println("  Usage: LinearRegression <data path> <result path> <num iterations>")
-		}
-		true
-	}
-
-	private def getDataSet(env: ExecutionEnvironment): DataSet[Data] = {
-		if (fileOutput) {
-			env.readCsvFile[(Double, Double)](
-				dataPath,
-				fieldDelimiter = ' ',
-				includedFields = Array(0, 1))
-				.map { t => new Data(t._1, t._2) }
-		}
-		else {
-			val data = LinearRegressionData.DATA map {
-				case Array(x, y) => Data(x.asInstanceOf[Double], y.asInstanceOf[Double])
-			}
-			env.fromCollection(data)
-		}
-	}
-
-	private def getParamsDataSet(env: ExecutionEnvironment): DataSet[Params] = {
-		val params = LinearRegressionData.PARAMS map {
-			case Array(x, y) => Params(x.asInstanceOf[Double], y.asInstanceOf[Double])
-		}
-		env.fromCollection(params)
-	}
+    }
+    true
+  }
+
+  private def getDataSet(env: ExecutionEnvironment): DataSet[Data] = {
+    if (fileOutput) {
+      env.readCsvFile[(Double, Double)](
+        dataPath,
+        fieldDelimiter = ' ',
+        includedFields = Array(0, 1))
+        .map { t => new Data(t._1, t._2) }
+    }
+    else {
+      val data = LinearRegressionData.DATA map {
+        case Array(x, y) => Data(x.asInstanceOf[Double], y.asInstanceOf[Double])
+      }
+      env.fromCollection(data)
+    }
+  }
+
+  private def getParamsDataSet(env: ExecutionEnvironment): DataSet[Params] = {
+    val params = LinearRegressionData.PARAMS map {
+      case Array(x, y) => Params(x.asInstanceOf[Double], y.asInstanceOf[Double])
+    }
+    env.fromCollection(params)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index e973093..0d0519c 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.api.scala
 
 import org.apache.commons.lang3.Validate
@@ -610,9 +609,9 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
       new Keys.FieldPositionKeys[T](fieldIndices, set.getType,false))
   }
 
-  //	public UnsortedGrouping<T> groupBy(String... fields) {
-  //		new UnsortedGrouping<T>(this, new Keys.ExpressionKeys<T>(fields, getType()));
-  //	}
+  //  public UnsortedGrouping<T> groupBy(String... fields) {
+  //    new UnsortedGrouping<T>(this, new Keys.ExpressionKeys<T>(fields, getType()));
+  //  }
 
   // --------------------------------------------------------------------------------------------
   //  Joining
@@ -807,7 +806,7 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
 
   /**
    * Creates a new DataSet by performing delta (or workset) iterations using the given step
-   * function. At the beginning `this` DataSet is the solution set and `workset` is  the Workset.
+   * function. At the beginning `this` DataSet is the solution set and `workset` is the Workset.
    * The iteration step function gets the current solution set and workset and must output the
    * delta for the solution set and the workset for the next iteration.
    *
@@ -825,6 +824,28 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
     wrap(result)
   }
 
+  /**
+   * Creates a new DataSet by performing delta (or workset) iterations using the given step
+   * function. At the beginning `this` DataSet is the solution set and `workset` is the Workset.
+   * The iteration step function gets the current solution set and workset and must output the
+   * delta for the solution set and the workset for the next iteration.
+   *
+   * Note: The syntax of delta iterations are very likely going to change soon.
+   */
+  def iterateDelta[R: ClassTag](workset: DataSet[R], maxIterations: Int, keyFields: Array[String])(
+    stepFunction: (DataSet[T], DataSet[R]) => (DataSet[T], DataSet[R])) = {
+    val fieldIndices = fieldNames2Indices(set.getType, keyFields)
+
+    val key = new FieldPositionKeys[T](fieldIndices, set.getType, false)
+    val iterativeSet = new DeltaIteration[T, R](
+      set.getExecutionEnvironment, set.getType, set, workset.set, key, maxIterations)
+    val (newSolution, newWorkset) = stepFunction(
+      wrap(iterativeSet.getSolutionSet),
+      wrap(iterativeSet.getWorkset))
+    val result = iterativeSet.closeWith(newSolution.set, newWorkset.set)
+    wrap(result)
+  }
+
   // -------------------------------------------------------------------------------------------
   //  Custom Operators
   // -------------------------------------------------------------------------------------------
@@ -919,4 +940,4 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
   def printToErr(): DataSink[T] = {
     output(new PrintingOutputFormat[T](true))
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
index 6f44e68..30b4525 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -22,7 +22,7 @@ import java.util.UUID
 import org.apache.commons.lang3.Validate
 import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.api.java.io._
-import org.apache.flink.api.java.typeutils.{TupleTypeInfoBase, BasicTypeInfo}
+import org.apache.flink.api.java.typeutils.{ValueTypeInfo, TupleTypeInfoBase, BasicTypeInfo}
 import org.apache.flink.api.scala.operators.ScalaCsvInputFormat
 import org.apache.flink.core.fs.Path
 
@@ -30,7 +30,7 @@ import org.apache.flink.api.java.{ExecutionEnvironment => JavaEnv}
 import org.apache.flink.api.common.io.{InputFormat, FileInputFormat}
 
 import org.apache.flink.api.java.operators.DataSource
-import org.apache.flink.types.TypeInformation
+import org.apache.flink.types.{StringValue, TypeInformation}
 import org.apache.flink.util.{NumberSequenceIterator, SplittableIterator}
 
 import scala.collection.JavaConverters._
@@ -105,6 +105,27 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
   }
 
   /**
+   * Creates a DataSet of Strings produced by reading the given file line wise.
+   * This method is similar to [[readTextFile]], but it produces a DataSet with mutable
+   * [[StringValue]] objects, rather than Java Strings. StringValues can be used to tune
+   * implementations to be less object and garbage collection heavy.
+   *
+   * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or
+   *                 "hdfs://host:port/file/path").
+   * @param charsetName The name of the character set used to read the file. Default is UTF-0
+   */
+  def readTextFileWithValue(
+      filePath: String,
+      charsetName: String = "UTF-8"): DataSet[StringValue] = {
+    Validate.notNull(filePath, "The file path may not be null.")
+    val format = new TextValueInputFormat(new Path(filePath))
+    format.setCharsetName(charsetName)
+    val source = new DataSource[StringValue](
+      javaEnv, format, new ValueTypeInfo[StringValue](classOf[StringValue]))
+    wrap(source)
+  }
+
+  /**
    * Creates a DataSet by reading the given CSV file. The type parameter must be used to specify
    * a Tuple type that has the same number of fields as there are fields in the CSV file. If the
    * number of fields in the CSV file is not the same, the `includedFields` parameter can be used
@@ -337,8 +358,9 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
   def createProgramPlan(jobName: String = "") = {
     if (jobName.isEmpty) {
       javaEnv.createProgramPlan()
-    } else
+    } else {
       javaEnv.createProgramPlan(jobName)
+    }
   }
 }
 
@@ -360,7 +382,8 @@ object ExecutionEnvironment {
    * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads).
    */
   def createLocalEnvironment(
-      degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()) : ExecutionEnvironment = {
+      degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors())
+      : ExecutionEnvironment = {
     val javaEnv = JavaEnv.createLocalEnvironment()
     javaEnv.setDegreeOfParallelism(degreeOfParallelism)
     new ExecutionEnvironment(javaEnv)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
index 802fd09..3c5bf9e 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
@@ -267,7 +267,7 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
   }
 
   def reduceGroup[R: TypeInformation: ClassTag](
-                                                 fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
+      fun: (TraversableOnce[T], Collector[R]) => Unit): DataSet[R] = {
     Validate.notNull(fun, "Group reduce function must not be null.")
     val reducer = new GroupReduceFunction[T, R] {
       def reduce(in: java.lang.Iterable[T], out: Collector[R]) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
index b0661a1..582edac 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
@@ -33,8 +33,8 @@ import scala.reflect.ClassTag
 
 
 /**
- * A specific [[DataSet]] that results from a `coGroup` operation. The result of a default coGroup is
- * a tuple containing two arrays of values from the two sides of the coGroup. The result of the
+ * A specific [[DataSet]] that results from a `coGroup` operation. The result of a default coGroup
+ * is a tuple containing two arrays of values from the two sides of the coGroup. The result of the
  * coGroup can be changed by specifying a custom coGroup function using the `apply` method or by
  * providing a [[RichCoGroupFunction]].
  *

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
index d538188..caa4324 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Counter.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.api.scala.codegen
 
 private[flink] class Counter {
@@ -29,4 +27,4 @@ private[flink] class Counter {
       current
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
index 4ce4922..9f17691 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/MacroContextHolder.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -24,8 +24,8 @@ private[flink] class MacroContextHolder[C <: Context](val c: C)
 
 private[flink] object MacroContextHolder {
   def newMacroHelper[C <: Context](c: C) = new MacroContextHolder[c.type](c)
-    	with TypeDescriptors[c.type]
-    	with TypeAnalyzer[c.type]
-    	with TreeGen[c.type]
+      with TypeDescriptors[c.type]
+      with TypeAnalyzer[c.type]
+      with TreeGen[c.type]
       with TypeInformationGen[c.type]
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
index 89454d5..7d04b1b 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TreeGen.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.api.scala.codegen
 
 import scala.language.implicitConversions
@@ -50,10 +48,11 @@ private[flink] trait TreeGen[C <: Context] { this: MacroContextHolder[C] with Ty
       reify(c.Expr(source).splice.asInstanceOf[T]).tree
 
     def maybeMkAsInstanceOf[S: c.WeakTypeTag, T: c.WeakTypeTag](source: Tree): Tree = {
-      if (weakTypeOf[S] <:< weakTypeOf[T])
+      if (weakTypeOf[S] <:< weakTypeOf[T]) {
         source
-      else
+      } else {
         mkAsInstanceOf[T](source)
+      }
     }
 
 //    def mkIdent(target: Symbol): Tree = Ident(target) setType target.tpe

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
index 7b1675d..6bf827e 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.api.scala.codegen
 
 import scala.Option.option2Iterable
@@ -107,10 +105,11 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
             appliedType(d.asType.toType, dArgs)
           }
 
-        if (dTpe <:< tpe)
+        if (dTpe <:< tpe) {
           Some(analyze(dTpe))
-        else
+        } else {
           None
+        }
       }
 
       val errors = subTypes flatMap { _.findByType[UnsupportedDescriptor] }
@@ -150,7 +149,11 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
                 case true =>
                   Some(
                     FieldAccessor(
-                      bGetter, bSetter, bTpe, isBaseField = true, analyze(bTpe.termSymbol.asMethod.returnType)))
+                      bGetter,
+                      bSetter,
+                      bTpe,
+                      isBaseField = true,
+                      analyze(bTpe.termSymbol.asMethod.returnType)))
                 case false => None
               }
           }
@@ -167,7 +170,9 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
 
             desc match {
               case desc @ BaseClassDescriptor(_, _, getters, baseSubTypes) =>
-                desc.copy(getters = getters map updateField, subTypes = baseSubTypes map wireBaseFields)
+                desc.copy(
+                  getters = getters map updateField,
+                  subTypes = baseSubTypes map wireBaseFields)
               case desc @ CaseClassDescriptor(_, _, _, _, getters) =>
                 desc.copy(getters = getters map updateField)
               case _ => desc
@@ -221,7 +226,7 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C]
                 case errs @ _ :: _ =>
                   val msgs = errs flatMap { f =>
                     (f: @unchecked) match {
-                      case FieldAccessor(fgetter, _, _, _, UnsupportedDescriptor(_, fTpe, errors)) =>
+                      case FieldAccessor(fgetter, _,_,_, UnsupportedDescriptor(_, fTpe, errors)) =>
                         errors map { err => "Field " + fgetter.name + ": " + fTpe + " - " + err }
                     }
                   }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
index bbb9e73..34a7a97 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.api.scala.codegen
 
 import scala.language.postfixOps
@@ -122,7 +120,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
       id: Int, tpe: Type, override val getters: Seq[FieldAccessor], subTypes: Seq[UDTDescriptor])
     extends UDTDescriptor {
 
-    override def flatten = this +: ((getters flatMap { _.desc.flatten }) ++ (subTypes flatMap { _.flatten }))
+    override def flatten =
+      this +: ((getters flatMap { _.desc.flatten }) ++ (subTypes flatMap { _.flatten }))
     override def canBeKey = flatten forall { f => f.canBeKey }
     
     override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
@@ -151,7 +150,8 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
     override def hashCode = (id, tpe, ctor, getters).hashCode
     override def equals(that: Any) = that match {
       case CaseClassDescriptor(thatId, thatTpe, thatMutable, thatCtor, thatGetters) =>
-        (id, tpe, mutable, ctor, getters).equals(thatId, thatTpe, thatMutable, thatCtor, thatGetters)
+        (id, tpe, mutable, ctor, getters).equals(
+          thatId, thatTpe, thatMutable, thatCtor, thatGetters)
       case _ => false
     }
     
@@ -164,7 +164,12 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C]
     }
   }
 
-  case class FieldAccessor(getter: Symbol, setter: Symbol, tpe: Type, isBaseField: Boolean, desc: UDTDescriptor)
+  case class FieldAccessor(
+      getter: Symbol,
+      setter: Symbol,
+      tpe: Type,
+      isBaseField: Boolean,
+      desc: UDTDescriptor)
 
   case class RecursiveDescriptor(id: Int, tpe: Type, refId: Int) extends UDTDescriptor {
     override def flatten = Seq(this)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index 6ad1f74..235caa7 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,8 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.api.scala.codegen
 
 import org.apache.flink.api.common.typeutils.TypeSerializer
@@ -116,14 +114,16 @@ private[flink] trait TypeInformationGen[C <: Context] {
     }
   }
 
-  def mkValueTypeInfo[T <: Value : c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
+  def mkValueTypeInfo[T <: Value : c.WeakTypeTag](
+      desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
     val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
     reify {
       new ValueTypeInfo[T](tpeClazz.splice)
     }
   }
 
-  def mkWritableTypeInfo[T <: Writable : c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
+  def mkWritableTypeInfo[T <: Writable : c.WeakTypeTag](
+      desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
     val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
     reify {
       new WritableTypeInfo[T](tpeClazz.splice)
@@ -153,7 +153,8 @@ private[flink] trait TypeInformationGen[C <: Context] {
     c.Expr[T](result)
   }
 
-//    def mkCaseClassTypeInfo[T: c.WeakTypeTag](desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = {
+//    def mkCaseClassTypeInfo[T: c.WeakTypeTag](
+//        desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = {
 //      val tpeClazz = c.Expr[Class[_]](Literal(Constant(desc.tpe)))
 //      val caseFields = mkCaseFields(desc)
 //      reify {
@@ -178,10 +179,12 @@ private[flink] trait TypeInformationGen[C <: Context] {
 //    c.Expr(mkMap(fields))
 //  }
 //
-//  protected def getFields(name: String, desc: UDTDescriptor): Seq[(String, UDTDescriptor)] = desc match {
+//  protected def getFields(name: String, desc: UDTDescriptor): Seq[(String, UDTDescriptor)] =
+//  desc match {
 //    // Flatten product types
 //    case CaseClassDescriptor(_, _, _, _, getters) =>
-//      getters filterNot { _.isBaseField } flatMap { f => getFields(name + "." + f.getter.name, f.desc) }
+//      getters filterNot { _.isBaseField } flatMap {
+//        f => getFields(name + "." + f.getter.name, f.desc) }
 //    case _ => Seq((name, desc))
 //  }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
index 2db2ff6..df34587 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
@@ -129,4 +129,4 @@ private[flink] object CrossDataSetImpl {
 
     new CrossDataSetImpl(crossOperator, leftSet, rightSet)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index c3f4dda..a607301 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -228,4 +228,4 @@ private[flink] class UnfinishedJoinOperationImpl[T, O](
 
     new JoinDataSetImpl(joinOperator, leftSet.set, rightSet.set, leftKey, rightKey)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
index c63c991..73dd721 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -44,4 +44,4 @@ package object scala {
           "supported on Case Classes (for now).")
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
index 3d21f05..1c8f8df 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
index a63bb35..90d3b5b 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.typeutils.TypeSerializer

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
index c191e81..ad407cb 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.java.typeutils.{AtomicType, TupleTypeInfoBase}
@@ -79,8 +78,8 @@ abstract class ScalaTupleTypeInfo[T <: Product](
   def getFieldIndices(fields: Array[String]): Array[Int] = {
     val result = fields map { x => fieldNames.indexOf(x) }
     if (result.contains(-1)) {
-      throw new IllegalArgumentException("Fields '" + fields.mkString(", ") + "' are not valid for" +
-        " " + tupleClass + " with fields '" + fieldNames.mkString(", ") + "'.")
+      throw new IllegalArgumentException("Fields '" + fields.mkString(", ") +
+        "' are not valid for " + tupleClass + " with fields '" + fieldNames.mkString(", ") + "'.")
     }
     result
   }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
index 5901e48..8d8f722 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.api.scala.typeutils
 
 import scala.reflect.macros.Context

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
index 198388e..8541481 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
@@ -71,7 +71,9 @@ private[flink] abstract class UnfinishedKeyPairOperation[T, O, R](
    * This only works on a CaseClass [[DataSet]].
    */
   def where(firstLeftField: String, otherLeftFields: String*) = {
-    val fieldIndices = fieldNames2Indices(leftSet.set.getType, firstLeftField +: otherLeftFields.toArray)
+    val fieldIndices = fieldNames2Indices(
+      leftSet.set.getType,
+      firstLeftField +: otherLeftFields.toArray)
 
     val leftKey = new FieldPositionKeys[T](fieldIndices, leftSet.set.getType)
     new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
index 5bfa9b4..5c4606f 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
@@ -1,12 +1,13 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -14,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.flink.api.scala
 
 import java.lang.reflect.Method
@@ -149,7 +149,8 @@ class ScalaAPICompletenessTest {
     checkMethods("SingleInputOperator", "DataSet",
       classOf[SingleInputOperator[_, _, _]], classOf[DataSet[_]])
 
-    checkMethods("TwoInputOperator", "DataSet", classOf[TwoInputOperator[_, _, _, _]], classOf[DataSet[_]])
+    checkMethods("TwoInputOperator", "DataSet",
+      classOf[TwoInputOperator[_, _, _, _]], classOf[DataSet[_]])
 
     checkMethods("SingleInputUdfOperator", "DataSet",
       classOf[SingleInputUdfOperator[_, _, _]], classOf[DataSet[_]])

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
index 0769891..29235d6 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/functions/SemanticPropertiesTranslationTest.scala
@@ -46,7 +46,7 @@ class SemanticPropertiesTranslationTest {
     try {
       val env = ExecutionEnvironment.getExecutionEnvironment
 
-      val input = env.fromElements((3l, "test", 42))
+      val input = env.fromElements((3L, "test", 42))
       input.map(new WildcardConstantMapper[(Long, String, Int)]).print()
 
       val plan = env.createProgramPlan()
@@ -83,7 +83,7 @@ class SemanticPropertiesTranslationTest {
     try {
       val env = ExecutionEnvironment.getExecutionEnvironment
 
-      val input = env.fromElements((3l, "test", 42))
+      val input = env.fromElements((3L, "test", 42))
       input.map(new IndividualConstantMapper[Long, String, Int]).print()
 
       val plan = env.createProgramPlan()
@@ -120,8 +120,8 @@ class SemanticPropertiesTranslationTest {
     try {
       val env = ExecutionEnvironment.getExecutionEnvironment
 
-      val input1 = env.fromElements((3l, "test"))
-      val input2 = env.fromElements((3l, 3.1415))
+      val input1 = env.fromElements((3L, "test"))
+      val input2 = env.fromElements((3L, 3.1415))
 
       input1.join(input2).where(0).equalTo(0)(
         new ForwardingTupleJoin[Long, String, Long, Double]).print()

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
index 9b953ee..c458e56 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
@@ -100,7 +100,9 @@ class DeltaIterationTranslationTest {
       assertEquals(classOf[IdentityMapper[_]], worksetMapper.getUserCodeWrapper.getUserCodeClass)
 
 
-      assertEquals(classOf[NextWorksetMapper], nextWorksetMapper.getUserCodeWrapper.getUserCodeClass)
+      assertEquals(
+        classOf[NextWorksetMapper],
+        nextWorksetMapper.getUserCodeWrapper.getUserCodeClass)
 
 
       if (solutionSetJoin.getUserCodeWrapper.getUserCodeObject.isInstanceOf[WrappingFunction[_]]) {
@@ -203,7 +205,8 @@ class DeltaIterationTranslationTest {
 //      val iteration: DeltaIteration[Tuple3[Double, Long, String], Tuple2[Double,
 //        String]] = initialSolutionSet.iterateDelta(initialWorkSet, 10, 1)
 //      try {
-//        iteration.getWorkset.coGroup(iteration.getSolutionSet).where(1).equalTo(2).`with`(new DeltaIterationTranslationTest.SolutionWorksetCoGroup1)
+//        iteration.getWorkset.coGroup(iteration.getSolutionSet).where(1).equalTo(2).`with`(
+// new DeltaIterationTranslationTest.SolutionWorksetCoGroup1)
 //        fail("Accepted invalid program.")
 //      }
 //      catch {
@@ -211,7 +214,8 @@ class DeltaIterationTranslationTest {
 //        }
 //      }
 //      try {
-//        iteration.getSolutionSet.coGroup(iteration.getWorkset).where(2).equalTo(1).`with`(new DeltaIterationTranslationTest.SolutionWorksetCoGroup2)
+//        iteration.getSolutionSet.coGroup(iteration.getWorkset).where(2).equalTo(1).`with`(
+// new DeltaIterationTranslationTest.SolutionWorksetCoGroup2)
 //        fail("Accepted invalid program.")
 //      }
 //      catch {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
index 36aa4f2..e0a9c89 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
@@ -121,7 +121,9 @@ class ReduceTranslationTest {
       assertEquals(keyValueInfo, reducer.getOperatorInfo.getOutputType)
       assertEquals(keyValueInfo, keyProjector.getOperatorInfo.getInputType)
       assertEquals(initialData.set.getType, keyProjector.getOperatorInfo.getOutputType)
-      assertEquals(classOf[KeyExtractingMapper[_, _]], keyExtractor.getUserCodeWrapper.getUserCodeClass)
+      assertEquals(
+        classOf[KeyExtractingMapper[_, _]],
+        keyExtractor.getUserCodeWrapper.getUserCodeClass)
       assertTrue(keyExtractor.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
     }
     catch {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
index ae6f15d..84486e6 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala
@@ -34,8 +34,8 @@ class TupleSerializerTest {
 
   @Test
   def testTuple1Int(): Unit = {
-    val testTuples =
-      Array(Tuple1(42), Tuple1(1), Tuple1(0), Tuple1(-1), Tuple1(Int.MaxValue), Tuple1(Int.MinValue))
+    val testTuples = Array(Tuple1(42), Tuple1(1), Tuple1(0), Tuple1(-1), Tuple1(Int.MaxValue),
+      Tuple1(Int.MinValue))
     runTests(testTuples)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6c42457..a339702 100644
--- a/pom.xml
+++ b/pom.xml
@@ -522,8 +522,10 @@ under the License.
 						<exclude>**/*.creole</exclude>
 						<exclude>CONTRIBUTORS</exclude>
 						<exclude>DEPENDENCIES</exclude>
-						<!-- Build fiels -->
+						<!-- Build files -->
 						<exclude>tools/maven/checkstyle.xml</exclude>
+						<exclude>tools/maven/scalastyle-config.xml</exclude>
+						<exclude>**/scalastyle-output.xml</exclude>
 						<exclude>tools/maven/suppressions.xml</exclude>
 						<exclude>**/pom.xml</exclude>
 						<exclude>**/pom.hadoop2.xml</exclude>
@@ -557,6 +559,29 @@ under the License.
 				</configuration>
 			</plugin>
 			<plugin>
+            	<groupId>org.scalastyle</groupId>
+           		<artifactId>scalastyle-maven-plugin</artifactId>
+            	<version>0.5.0</version>
+            	<configuration>
+              		<verbose>false</verbose>
+              		<failOnViolation>true</failOnViolation>
+              		<includeTestSourceDirectory>true</includeTestSourceDirectory>
+              		<failOnWarning>false</failOnWarning>
+              		<sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
+              		<testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
+              		<configLocation>tools/maven/scalastyle-config.xml</configLocation>
+              		<outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
+              		<outputEncoding>UTF-8</outputEncoding>
+            	</configuration>
+            	<executions>
+              		<execution>
+                		<goals>
+                  			<goal>check</goal>
+                		</goals>
+              		</execution>
+            	</executions>
+          </plugin>
+			<plugin>
 				<!-- just define the Java version to be used for compiling and plugins -->
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-compiler-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0385651f/tools/maven/scalastyle-config.xml
----------------------------------------------------------------------
diff --git a/tools/maven/scalastyle-config.xml b/tools/maven/scalastyle-config.xml
new file mode 100644
index 0000000..60d7418
--- /dev/null
+++ b/tools/maven/scalastyle-config.xml
@@ -0,0 +1,146 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<!-- NOTE: This was taken and adapted from Apache Spark. -->
+
+<!-- If you wish to turn off checking for a section of code, you can put a comment in the source
+ before and after the section, with the following syntax: -->
+<!-- // scalastyle:off -->
+<!-- ... -->
+<!-- // naughty stuff -->
+<!-- ... -->
+<!-- // scalastyle:on -->
+
+<scalastyle>
+ <name>Scalastyle standard configuration</name>
+ <check level="error" class="org.scalastyle.file.FileTabChecker" enabled="true"></check>
+ <!-- <check level="error" class="org.scalastyle.file.FileLengthChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="maxFileLength"><![CDATA[800]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <check level="error" class="org.scalastyle.file.HeaderMatchesChecker" enabled="true">
+  <parameters>
+      <parameter name="header"><![CDATA[/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */]]></parameter>
+  </parameters>
+ </check>
+ <check level="error" class="org.scalastyle.scalariform.SpacesAfterPlusChecker" enabled="true"></check>
+ <check level="error" class="org.scalastyle.file.WhitespaceEndOfLineChecker" enabled="false"></check>
+ <check level="error" class="org.scalastyle.scalariform.SpacesBeforePlusChecker" enabled="true"></check>
+ <check level="error" class="org.scalastyle.file.FileLineLengthChecker" enabled="true">
+  <parameters>
+   <parameter name="maxLineLength"><![CDATA[100]]></parameter>
+   <parameter name="tabSize"><![CDATA[2]]></parameter>
+   <parameter name="ignoreImports">true</parameter>
+  </parameters>
+ </check>
+ <check level="error" class="org.scalastyle.scalariform.ClassNamesChecker" enabled="true">
+  <parameters>
+   <parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
+  </parameters>
+ </check>
+ <check level="error" class="org.scalastyle.scalariform.ObjectNamesChecker" enabled="true">
+  <parameters>
+   <parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter>
+  </parameters>
+ </check>
+ <check level="error" class="org.scalastyle.scalariform.PackageObjectNamesChecker" enabled="true">
+  <parameters>
+   <parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter>
+  </parameters>
+ </check>
+ <check level="error" class="org.scalastyle.scalariform.EqualsHashCodeChecker" enabled="false"></check>
+ <!-- <check level="error" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <check level="error" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true">
+  <parameters>
+   <parameter name="maxParameters"><![CDATA[10]]></parameter>
+  </parameters>
+ </check>
+ <!-- <check level="error" class="org.scalastyle.scalariform.MagicNumberChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="ignore"><![CDATA[-1,0,1,2,3]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="false"></check>
+ <check level="error" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="false"></check>
+ <!-- <check level="error" class="org.scalastyle.scalariform.ReturnChecker" enabled="true"></check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.NullChecker" enabled="true"></check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.NoFinalizeChecker" enabled="true"></check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.CovariantEqualsChecker" enabled="true"></check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.StructuralTypeChecker" enabled="true"></check> -->
+ <!-- <check level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="regex"><![CDATA[println]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.NumberOfTypesChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="maxTypes"><![CDATA[30]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.CyclomaticComplexityChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="maximum"><![CDATA[10]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <check level="error" class="org.scalastyle.scalariform.UppercaseLChecker" enabled="true"></check>
+ <check level="error" class="org.scalastyle.scalariform.SimplifyBooleanExpressionChecker" enabled="false"></check>
+ <check level="error" class="org.scalastyle.scalariform.IfBraceChecker" enabled="true">
+  <parameters>
+   <parameter name="singleLineAllowed"><![CDATA[true]]></parameter>
+   <parameter name="doubleLineAllowed"><![CDATA[true]]></parameter>
+  </parameters>
+ </check>
+ <!-- <check level="error" class="org.scalastyle.scalariform.MethodLengthChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="maxLength"><![CDATA[50]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.MethodNamesChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="regex"><![CDATA[^[a-z][A-Za-z0-9]*$]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.NumberOfMethodsInTypeChecker" enabled="true"> -->
+ <!--  <parameters> -->
+ <!--   <parameter name="maxMethods"><![CDATA[30]]></parameter> -->
+ <!--  </parameters> -->
+ <!-- </check> -->
+ <!-- <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="true"></check> -->
+ <check level="error" class="org.scalastyle.file.NewLineAtEofChecker" enabled="true"></check>
+ <check level="error" class="org.scalastyle.file.NoNewLineAtEofChecker" enabled="false"></check>
+</scalastyle>


[35/60] git commit: Add logback-test.xml to java8-tests and logback.xml to java-examples

Posted by al...@apache.org.
Add logback-test.xml to java8-tests and logback.xml to java-examples


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

Branch: refs/heads/master
Commit: 81e81b9599e1d27fe056c7e5b53b65318fe6e687
Parents: dd9e27e
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Sep 11 16:14:32 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:58 2014 +0200

----------------------------------------------------------------------
 .../src/main/resources/logback.xml              | 29 ++++++++++++++++++++
 .../src/test/resources/logback-test.xml         | 29 ++++++++++++++++++++
 2 files changed, 58 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/81e81b95/flink-examples/flink-java-examples/src/main/resources/logback.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/resources/logback.xml b/flink-examples/flink-java-examples/src/main/resources/logback.xml
new file mode 100644
index 0000000..8b3bb27
--- /dev/null
+++ b/flink-examples/flink-java-examples/src/main/resources/logback.xml
@@ -0,0 +1,29 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/81e81b95/flink-java8-tests/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-java8-tests/src/test/resources/logback-test.xml b/flink-java8-tests/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..8b3bb27
--- /dev/null
+++ b/flink-java8-tests/src/test/resources/logback-test.xml
@@ -0,0 +1,29 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+</configuration>
\ No newline at end of file


[10/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
index 349c2c2..2ef029f 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRank.scala
@@ -1,108 +1,108 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.examples.scala.graph;
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-/**
- * An example program computing the page rank for each vertex in a graph.
- * The graph is initially represented by vertices and edges. Vertices are numeric identifiers, while
- * edges are pairs of identifiers that represent the source and target vertex.
- * 
- * This variant of page rank assumes that all edges that originate at one vertex have an equal
- * probability of being chosen.
- */
-class PageRank extends Program with Serializable {
-
-  def getScalaPlan(verticesPath: String, edgesPath: String, outputPath: String, numVertices: Long, maxIterations: Int) = {
-
-    case class PageWithRank(pageId: Long, rank: Double)
-    case class Edge(from: Long, to: Long)
-    case class Adjacency(vertex: Long, neighbors: List[Long])
-
-    // read the pages and edges. the pages are only single decimal identifiers, the edges pairs of identifiers
-    val pages = DataSource(verticesPath, CsvInputFormat[Long]())
-    val edges = DataSource(edgesPath, CsvInputFormat[Edge]("\n", ' '))
-
-    // some constants used in the specific rank computation
-    val dampening = 0.85
-    val randomJump = (1.0 - dampening) / numVertices
-    val initialRank = 1.0 / numVertices
-
-    // assign the initial uniform rank to all pages
-    val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
-    
-    // transform the edges from a list of (from -> target) pairs to an adjacency list (from -> [all-targets])
-    val adjacencies = edges.groupBy(_.from).reduceGroup(x => x.foldLeft(Adjacency(0, List[Long]()))((a, e) => Adjacency(e.from, e.to :: a.neighbors)));
-
-    def computeRank(ranks: DataSet[PageWithRank]) = {
-
-      val ranksForNeighbors = ranks join adjacencies where { _.pageId } isEqualTo { _.vertex } flatMap ( (p, e) => {
-        val numNeighbors = e.neighbors.length
-        
-        for (target <- e.neighbors)
-          yield (target, p.rank / numNeighbors)
-          
-      });
-
-      ranksForNeighbors.groupBy { case (node, rank) => node }
-        .reduce { (a, b) => (a._1, a._2 + b._2) }
-        .map { case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
-    }
-
-    val finalRanks = pagesWithRank.iterate(maxIterations, computeRank)
-
-    val output = finalRanks.write(outputPath, CsvOutputFormat())
-
-    new ScalaPlan(Seq(output), "Page Rank")
-  }
-
-  override def getPlan(args: String*) = {
-    val planArgs: Array[String] = if (args.length < 5) Array[String]("", "", "", "", "") else args.toArray
-    val dop = if (args.size > 5) args(5).toInt else 1
-
-    val plan = getScalaPlan(planArgs(0), planArgs(1), planArgs(2), planArgs(3).toLong, planArgs(4).toInt)
-    plan.setDefaultParallelism(dop)
-    plan
-  }
-}
-
-/**
- * Executable entry point to run the program locally.
- */
-object RunPageRank {
-
-  def main(args: Array[String]) {
-    if (args.size < 5) {
-      println("PageRank <pages input path> <links input path> <result path> <num pages> <num iterations> [<parallelism=1>]")
-      return
-    }
-
-    val dop = if (args.length > 5) args(5).toInt else 1
-    val plan = new PageRank().getScalaPlan(args(0), args(1), args(2), args(3).toLong, args(4).toInt);
-
-    plan.setDefaultParallelism(dop)
-    LocalExecutor.execute(plan)
-  }
-}
-
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.flink.examples.scala.graph;
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+///**
+// * An example program computing the page rank for each vertex in a graph.
+// * The graph is initially represented by vertices and edges. Vertices are numeric identifiers, while
+// * edges are pairs of identifiers that represent the source and target vertex.
+// *
+// * This variant of page rank assumes that all edges that originate at one vertex have an equal
+// * probability of being chosen.
+// */
+//class PageRank extends Program with Serializable {
+//
+//  def getScalaPlan(verticesPath: String, edgesPath: String, outputPath: String, numVertices: Long, maxIterations: Int) = {
+//
+//    case class PageWithRank(pageId: Long, rank: Double)
+//    case class Edge(from: Long, to: Long)
+//    case class Adjacency(vertex: Long, neighbors: List[Long])
+//
+//    // read the pages and edges. the pages are only single decimal identifiers, the edges pairs of identifiers
+//    val pages = DataSource(verticesPath, CsvInputFormat[Long]())
+//    val edges = DataSource(edgesPath, CsvInputFormat[Edge]("\n", ' '))
+//
+//    // some constants used in the specific rank computation
+//    val dampening = 0.85
+//    val randomJump = (1.0 - dampening) / numVertices
+//    val initialRank = 1.0 / numVertices
+//
+//    // assign the initial uniform rank to all pages
+//    val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
+//
+//    // transform the edges from a list of (from -> target) pairs to an adjacency list (from -> [all-targets])
+//    val adjacencies = edges.groupBy(_.from).reduceGroup(x => x.foldLeft(Adjacency(0, List[Long]()))((a, e) => Adjacency(e.from, e.to :: a.neighbors)));
+//
+//    def computeRank(ranks: DataSetOLD[PageWithRank]) = {
+//
+//      val ranksForNeighbors = ranks join adjacencies where { _.pageId } isEqualTo { _.vertex } flatMap ( (p, e) => {
+//        val numNeighbors = e.neighbors.length
+//
+//        for (target <- e.neighbors)
+//          yield (target, p.rank / numNeighbors)
+//
+//      });
+//
+//      ranksForNeighbors.groupBy { case (node, rank) => node }
+//        .reduce { (a, b) => (a._1, a._2 + b._2) }
+//        .map { case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
+//    }
+//
+//    val finalRanks = pagesWithRank.iterate(maxIterations, computeRank)
+//
+//    val output = finalRanks.write(outputPath, CsvOutputFormat())
+//
+//    new ScalaPlan(Seq(output), "Page Rank")
+//  }
+//
+//  override def getPlan(args: String*) = {
+//    val planArgs: Array[String] = if (args.length < 5) Array[String]("", "", "", "", "") else args.toArray
+//    val dop = if (args.size > 5) args(5).toInt else 1
+//
+//    val plan = getScalaPlan(planArgs(0), planArgs(1), planArgs(2), planArgs(3).toLong, planArgs(4).toInt)
+//    plan.setDefaultParallelism(dop)
+//    plan
+//  }
+//}
+//
+///**
+// * Executable entry point to run the program locally.
+// */
+//object RunPageRank {
+//
+//  def main(args: Array[String]) {
+//    if (args.size < 5) {
+//      println("PageRank <pages input path> <links input path> <result path> <num pages> <num iterations> [<parallelism=1>]")
+//      return
+//    }
+//
+//    val dop = if (args.length > 5) args(5).toInt else 1
+//    val plan = new PageRank().getScalaPlan(args(0), args(1), args(2), args(3).toLong, args(4).toInt);
+//
+//    plan.setDefaultParallelism(dop)
+//    LocalExecutor.execute(plan)
+//  }
+//}
+//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
index 44c8b61..97ee62e 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankWithWeight.scala
@@ -1,108 +1,108 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.examples.scala.graph;
-
-import scala.math._
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-import org.apache.flink.api.scala.analysis.GlobalSchemaPrinter
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-import org.apache.flink.api.common.Plan
-import org.apache.flink.api.java.record.operators.DeltaIteration
-
-/**
- * An implementation of the PageRank algorithm for graph vertex ranking. Runs a specified fix number
- * of iterations. This version of page rank expects the edges to define a transition
- * probability and hence allows to model situations where not all outgoing links are equally probable.
- * 
- * <p>
- * 
- * Expects inputs are:
- *  1. Path to a file of node ids, as a sequence of Longs, line delimited.
- *  2. Path to a csv file of edges in the format <tt>sourceId targetId transitionProbability</tt> (fields separated by spaces).
- *    The ids are expected to be Longs, the transition probability a float or double.
- *  3. Path to where the output should be written
- *  4. The number of vertices
- *  5. The number of iterations
- */
-class PageRankWithWeight extends Program with Serializable {
-
-  def getScalaPlan(verticesPath: String, edgesPath: String, outputPath: String, numVertices: Long, maxIterations: Int) = {
-
-    case class PageWithRank(pageId: Long, rank: Double)
-    case class Edge(from: Long, to: Long, transitionProb: Double)
-
-    val pages = DataSource(verticesPath, CsvInputFormat[Long]())
-    val edges = DataSource(edgesPath, CsvInputFormat[Edge]("\n", ' ')) // line delimiter (\n), field delimiter (' ')
-
-    val dampening = 0.85
-    val randomJump = (1.0 - dampening) / numVertices
-    val initialRank = 1.0 / numVertices
-
-    val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
-
-    def computeRank(ranks: DataSet[PageWithRank]) = {
-
-      val ranksForNeighbors = ranks join edges where { _.pageId } isEqualTo { _.from } map { (p, e) => (e.to, p.rank * e.transitionProb) }
-
-      ranksForNeighbors.groupBy { case (node, rank) => node }
-        .reduce { (a, b) => (a._1, a._2 + b._2) }
-        .map { case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
-    }
-
-    val finalRanks = pagesWithRank.iterate(maxIterations, computeRank)
-
-    val output = finalRanks.write(outputPath, CsvOutputFormat())
-
-    new ScalaPlan(Seq(output), "Connected Components")
-  }
-
-  override def getPlan(args: String*) = {
-    val planArgs: Array[String] = if (args.length < 5) Array[String]("", "", "", "", "") else args.toArray
-    val dop = if (args.size > 5) args(5).toInt else 1
-
-    val plan = getScalaPlan(planArgs(0), planArgs(1), planArgs(2), planArgs(3).toLong, planArgs(4).toInt)
-    plan.setDefaultParallelism(dop)
-    plan
-  }
-}
-
-/**
- * Executable entry point to run the program locally.
- */
-object RunPageRankWithWeight {
-
-  def main(args: Array[String]) {
-    if (args.size < 5) {
-      println("PageRank <vertices> <edges> <result> <numVertices> <numIterations> [<parallelism=1>]")
-      return
-    }
-
-    val dop = if (args.length > 5) args(5).toInt else 1
-    val plan = new PageRankWithWeight().getScalaPlan(args(0), args(1), args(2), args(3).toLong, args(4).toInt);
-
-    plan.setDefaultParallelism(dop)
-    LocalExecutor.execute(plan)
-  }
-}
-
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.flink.examples.scala.graph;
+//
+//import scala.math._
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//import org.apache.flink.api.scala.analysis.GlobalSchemaPrinter
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//import org.apache.flink.api.common.Plan
+//import org.apache.flink.api.java.record.operators.DeltaIteration
+//
+///**
+// * An implementation of the PageRank algorithm for graph vertex ranking. Runs a specified fix number
+// * of iterations. This version of page rank expects the edges to define a transition
+// * probability and hence allows to model situations where not all outgoing links are equally probable.
+// *
+// * <p>
+// *
+// * Expects inputs are:
+// *  1. Path to a file of node ids, as a sequence of Longs, line delimited.
+// *  2. Path to a csv file of edges in the format <tt>sourceId targetId transitionProbability</tt> (fields separated by spaces).
+// *    The ids are expected to be Longs, the transition probability a float or double.
+// *  3. Path to where the output should be written
+// *  4. The number of vertices
+// *  5. The number of iterations
+// */
+//class PageRankWithWeight extends Program with Serializable {
+//
+//  def getScalaPlan(verticesPath: String, edgesPath: String, outputPath: String, numVertices: Long, maxIterations: Int) = {
+//
+//    case class PageWithRank(pageId: Long, rank: Double)
+//    case class Edge(from: Long, to: Long, transitionProb: Double)
+//
+//    val pages = DataSource(verticesPath, CsvInputFormat[Long]())
+//    val edges = DataSource(edgesPath, CsvInputFormat[Edge]("\n", ' ')) // line delimiter (\n), field delimiter (' ')
+//
+//    val dampening = 0.85
+//    val randomJump = (1.0 - dampening) / numVertices
+//    val initialRank = 1.0 / numVertices
+//
+//    val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
+//
+//    def computeRank(ranks: DataSetOLD[PageWithRank]) = {
+//
+//      val ranksForNeighbors = ranks join edges where { _.pageId } isEqualTo { _.from } map { (p, e) => (e.to, p.rank * e.transitionProb) }
+//
+//      ranksForNeighbors.groupBy { case (node, rank) => node }
+//        .reduce { (a, b) => (a._1, a._2 + b._2) }
+//        .map { case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
+//    }
+//
+//    val finalRanks = pagesWithRank.iterate(maxIterations, computeRank)
+//
+//    val output = finalRanks.write(outputPath, CsvOutputFormat())
+//
+//    new ScalaPlan(Seq(output), "Connected Components")
+//  }
+//
+//  override def getPlan(args: String*) = {
+//    val planArgs: Array[String] = if (args.length < 5) Array[String]("", "", "", "", "") else args.toArray
+//    val dop = if (args.size > 5) args(5).toInt else 1
+//
+//    val plan = getScalaPlan(planArgs(0), planArgs(1), planArgs(2), planArgs(3).toLong, planArgs(4).toInt)
+//    plan.setDefaultParallelism(dop)
+//    plan
+//  }
+//}
+//
+///**
+// * Executable entry point to run the program locally.
+// */
+//object RunPageRankWithWeight {
+//
+//  def main(args: Array[String]) {
+//    if (args.size < 5) {
+//      println("PageRank <vertices> <edges> <result> <numVertices> <numIterations> [<parallelism=1>]")
+//      return
+//    }
+//
+//    val dop = if (args.length > 5) args(5).toInt else 1
+//    val plan = new PageRankWithWeight().getScalaPlan(args(0), args(1), args(2), args(3).toLong, args(4).toInt);
+//
+//    plan.setDefaultParallelism(dop)
+//    LocalExecutor.execute(plan)
+//  }
+//}
+//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
index 5c7beaf..858ce30 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
@@ -1,87 +1,87 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.graph;
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-class TransitiveClosureNaive extends Program with ProgramDescription with Serializable {
-
-  def getScalaPlan(numSubTasks: Int, numIterations: Int, verticesInput: String, edgesInput: String, pathsOutput: String) = {
-    val vertices = DataSource(verticesInput, DelimitedInputFormat(parseVertex))
-    val edges = DataSource(edgesInput, DelimitedInputFormat(parseEdge))
-
-    def createClosure(paths: DataSet[Path]) = {
-
-      val allNewPaths = paths join edges where { p => p.to } isEqualTo { e => e.from } map joinPaths
-      val shortestPaths = allNewPaths union paths groupBy { p => (p.from, p.to) } reduceGroup { _ minBy { _.dist } }
-
-      shortestPaths
-    }
-
-    val transitiveClosure = vertices.iterate(numIterations, createClosure)
-
-    val output = transitiveClosure.write(pathsOutput, DelimitedOutputFormat(formatOutput))
-
-    val plan = new ScalaPlan(Seq(output), "Transitive Closure (Naive)")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-
-  def joinPaths = (p1: Path, p2: Path) => (p1, p2) match {
-      case (Path(from, _, dist1), Path(_, to, dist2)) => Path(from, to, dist1 + dist2)
-  }
-
-  case class Path(from: Int, to: Int, dist: Int)
-
-  def parseVertex = (line: String) => { val v = line.toInt; Path(v, v, 0) }
-
-  val EdgeInputPattern = """(\d+)\|(\d+)""".r
-
-  def parseEdge = (line: String) => line match {
-    case EdgeInputPattern(from, to) => Path(from.toInt, to.toInt, 1)
-  }
-
-  def formatOutput = (path: Path) => "%d|%d|%d".format(path.from, path.to, path.dist)
-
-  override def getDescription() = {
-    "Parameters: <numSubStasks> <numIterations> <vertices> <edges> <output>"
-  }
-
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1).toInt, args(2), args(3), args(4))
-  }
-}
-
-object RunTransitiveClosureNaive {
-  def main(pArgs: Array[String]) {
-    if (pArgs.size < 3) {
-      println("usage: [-numIterations <int:2>] -vertices <file> -edges <file> -output <file>")
-      return
-    }
-    val args = Args.parse(pArgs)
-    val plan = new TransitiveClosureNaive().getScalaPlan(2, args("numIterations", "10").toInt, args("vertices"), args("edges"), args("output"))
-    LocalExecutor.execute(plan)
-  }
-}
\ No newline at end of file
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.graph;
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//class TransitiveClosureNaive extends Program with ProgramDescription with Serializable {
+//
+//  def getScalaPlan(numSubTasks: Int, numIterations: Int, verticesInput: String, edgesInput: String, pathsOutput: String) = {
+//    val vertices = DataSource(verticesInput, DelimitedInputFormat(parseVertex))
+//    val edges = DataSource(edgesInput, DelimitedInputFormat(parseEdge))
+//
+//    def createClosure(paths: DataSetOLD[Path]) = {
+//
+//      val allNewPaths = paths join edges where { p => p.to } isEqualTo { e => e.from } map joinPaths
+//      val shortestPaths = allNewPaths union paths groupBy { p => (p.from, p.to) } reduceGroup { _ minBy { _.dist } }
+//
+//      shortestPaths
+//    }
+//
+//    val transitiveClosure = vertices.iterate(numIterations, createClosure)
+//
+//    val output = transitiveClosure.write(pathsOutput, DelimitedOutputFormat(formatOutput))
+//
+//    val plan = new ScalaPlan(Seq(output), "Transitive Closure (Naive)")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//
+//  def joinPaths = (p1: Path, p2: Path) => (p1, p2) match {
+//      case (Path(from, _, dist1), Path(_, to, dist2)) => Path(from, to, dist1 + dist2)
+//  }
+//
+//  case class Path(from: Int, to: Int, dist: Int)
+//
+//  def parseVertex = (line: String) => { val v = line.toInt; Path(v, v, 0) }
+//
+//  val EdgeInputPattern = """(\d+)\|(\d+)""".r
+//
+//  def parseEdge = (line: String) => line match {
+//    case EdgeInputPattern(from, to) => Path(from.toInt, to.toInt, 1)
+//  }
+//
+//  def formatOutput = (path: Path) => "%d|%d|%d".format(path.from, path.to, path.dist)
+//
+//  override def getDescription() = {
+//    "Parameters: <numSubStasks> <numIterations> <vertices> <edges> <output>"
+//  }
+//
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1).toInt, args(2), args(3), args(4))
+//  }
+//}
+//
+//object RunTransitiveClosureNaive {
+//  def main(pArgs: Array[String]) {
+//    if (pArgs.size < 3) {
+//      println("usage: [-numIterations <int:2>] -vertices <file> -edges <file> -output <file>")
+//      return
+//    }
+//    val args = Args.parse(pArgs)
+//    val plan = new TransitiveClosureNaive().getScalaPlan(2, args("numIterations", "10").toInt, args("vertices"), args("edges"), args("output"))
+//    LocalExecutor.execute(plan)
+//  }
+//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
index 8ba71d8..b9f2264 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
@@ -1,78 +1,78 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.iterative
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-/**
- * Example of using the bulk iteration with termination criterion with the
- * scala api.
- */
-class TerminationCriterion extends Program with ProgramDescription with Serializable {
-  override def getDescription() = {
-    "Parameters: <maxNumberIterations> <output>"
-  }
-
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1))
-  }
-
-  def getScalaPlan(maxNumberIterations: Int, resultOutput: String) = {
-    val dataSource = CollectionDataSource[Double](List(1.0))
-
-    val halve = (partialSolution: DataSet[Double]) => {
-      partialSolution map { x => x /2 }
-    }
-
-    val terminationCriterion = (prev: DataSet[Double], cur: DataSet[Double]) => {
-      val diff = prev cross cur map { (valuePrev, valueCurrent) => math.abs(valuePrev - valueCurrent) }
-      diff filter {
-        difference => difference > 0.1
-      }
-    }
-
-    val iteration = dataSource.iterateWithTermination(maxNumberIterations, halve, terminationCriterion)
-
-
-    val sink = iteration.write(resultOutput, CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(sink))
-    plan.setDefaultParallelism(1)
-    plan
-  }
-}
-
-object RunTerminationCriterion {
-  def main(args: Array[String]) {
-    val tc = new TerminationCriterion
-
-    if(args.size < 2) {
-      println(tc.getDescription())
-      return
-    }
-    val plan = tc.getScalaPlan(args(0).toInt, args(1))
-    LocalExecutor.execute(plan)
-  }
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.iterative
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+///**
+// * Example of using the bulk iteration with termination criterion with the
+// * scala api.
+// */
+//class TerminationCriterion extends Program with ProgramDescription with Serializable {
+//  override def getDescription() = {
+//    "Parameters: <maxNumberIterations> <output>"
+//  }
+//
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1))
+//  }
+//
+//  def getScalaPlan(maxNumberIterations: Int, resultOutput: String) = {
+//    val dataSource = CollectionDataSource[Double](List(1.0))
+//
+//    val halve = (partialSolution: DataSetOLD[Double]) => {
+//      partialSolution map { x => x /2 }
+//    }
+//
+//    val terminationCriterion = (prev: DataSetOLD[Double], cur: DataSetOLD[Double]) => {
+//      val diff = prev cross cur map { (valuePrev, valueCurrent) => math.abs(valuePrev - valueCurrent) }
+//      diff filter {
+//        difference => difference > 0.1
+//      }
+//    }
+//
+//    val iteration = dataSource.iterateWithTermination(maxNumberIterations, halve, terminationCriterion)
+//
+//
+//    val sink = iteration.write(resultOutput, CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(sink))
+//    plan.setDefaultParallelism(1)
+//    plan
+//  }
+//}
+//
+//object RunTerminationCriterion {
+//  def main(args: Array[String]) {
+//    val tc = new TerminationCriterion
+//
+//    if(args.size < 2) {
+//      println(tc.getDescription())
+//      return
+//    }
+//    val plan = tc.getScalaPlan(args(0).toInt, args(1))
+//    LocalExecutor.execute(plan)
+//  }
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
index 9a67ee4..5e7d7f3 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
@@ -1,107 +1,107 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.relational;
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-
-/**
- * The TPC-H is a decision support benchmark on relational data.
- * Its documentation and the data generator (DBGEN) can be found
- * on http://www.tpc.org/tpch/ .
- * 
- * This Flink program implements a modified version of the query 3 of
- * the TPC-H benchmark including one join, some filtering and an
- * aggregation. The query resembles the following SQL statement:
- * <pre>
- * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
- *   FROM orders, lineitem
- *   WHERE l_orderkey = o_orderkey
- *     AND o_orderstatus = "X"
- *     AND YEAR(o_orderdate) > Y
- *     AND o_orderpriority LIKE "Z%"
- *   GROUP BY l_orderkey, o_shippriority;
- * </pre>
- */
-class RelationalQuery extends Program with ProgramDescription with Serializable {
-
-  case class Order(orderId: Int, status: Char, year: Int, orderPriority: String, shipPriority: Int)
-  case class LineItem(orderId: Int, extendedPrice: Double)
-  case class PrioritizedOrder(orderId: Int, shipPriority: Int, revenue: Double)
-  
-  
-  def getScalaPlan(numSubTasks: Int, ordersInput: String, lineItemsInput: String, ordersOutput: String, status: Char = 'F', minYear: Int = 1993, priority: String = "5") = {
-    
-    // ORDER intput: parse as CSV and select relevant fields
-    val orders = DataSource(ordersInput, CsvInputFormat[(Int, String, String, String, String, String, String, Int)]("\n", '|'))
-                         .map { t => Order(t._1, t._3.charAt(0), t._5.substring(0,4).toInt, t._6, t._8) }
-      
-    // ORDER intput: parse as CSV and select relevant fields
-    val lineItems = DataSource(lineItemsInput, CsvInputFormat[(Int, String, String, String, String, Double)]("\n", '|'))
-                         .map { t => LineItem(t._1, t._6) }
-    
-    // filter the orders input
-    val filteredOrders = orders filter { o => o.status == status && o.year > minYear && o.orderPriority.startsWith(priority) }
-    
-    // join the filteres result with the lineitem input
-    val prioritizedItems = filteredOrders join lineItems where { _.orderId } isEqualTo { _.orderId } map { (o, li) => PrioritizedOrder(o.orderId, o.shipPriority, li.extendedPrice) }
-    
-    // group by and sum the joined data
-    val prioritizedOrders = prioritizedItems groupBy { pi => (pi.orderId, pi.shipPriority) } reduce { (po1, po2) => po1.copy(revenue = po1.revenue + po2.revenue) }
-
-    // write the result as csv
-    val output = prioritizedOrders.write(ordersOutput, CsvOutputFormat("\n", "|"))
-
-    val plan = new ScalaPlan(Seq(output), "Relational Query")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-
-  override def getDescription() = {
-    "Parameters: <orders>, <lineitem>, <output>, <degree-of-parallelism>"
-  }
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(3).toInt, args(0), args(1), args(2))
-  }
-}
-
-
-/**
- * Entry point to make the example standalone runnable with the local executor
- */
-object RunRelationalQuery {
-  
-  def main(args: Array[String]) {
-    val query = new RelationalQuery
-    
-    if (args.size < 4) {
-      println(query.getDescription)
-      return
-    }
-    val plan = query.getScalaPlan(args(3).toInt, args(0), args(1), args(2))
-    LocalExecutor.execute(plan)
-  }
-}
-
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.relational;
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//
+///**
+// * The TPC-H is a decision support benchmark on relational data.
+// * Its documentation and the data generator (DBGEN) can be found
+// * on http://www.tpc.org/tpch/ .
+// *
+// * This Flink program implements a modified version of the query 3 of
+// * the TPC-H benchmark including one join, some filtering and an
+// * aggregation. The query resembles the following SQL statement:
+// * <pre>
+// * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+// *   FROM orders, lineitem
+// *   WHERE l_orderkey = o_orderkey
+// *     AND o_orderstatus = "X"
+// *     AND YEAR(o_orderdate) > Y
+// *     AND o_orderpriority LIKE "Z%"
+// *   GROUP BY l_orderkey, o_shippriority;
+// * </pre>
+// */
+//class RelationalQuery extends Program with ProgramDescription with Serializable {
+//
+//  case class Order(orderId: Int, status: Char, year: Int, orderPriority: String, shipPriority: Int)
+//  case class LineItem(orderId: Int, extendedPrice: Double)
+//  case class PrioritizedOrder(orderId: Int, shipPriority: Int, revenue: Double)
+//
+//
+//  def getScalaPlan(numSubTasks: Int, ordersInput: String, lineItemsInput: String, ordersOutput: String, status: Char = 'F', minYear: Int = 1993, priority: String = "5") = {
+//
+//    // ORDER intput: parse as CSV and select relevant fields
+//    val orders = DataSource(ordersInput, CsvInputFormat[(Int, String, String, String, String, String, String, Int)]("\n", '|'))
+//                         .map { t => Order(t._1, t._3.charAt(0), t._5.substring(0,4).toInt, t._6, t._8) }
+//
+//    // ORDER intput: parse as CSV and select relevant fields
+//    val lineItems = DataSource(lineItemsInput, CsvInputFormat[(Int, String, String, String, String, Double)]("\n", '|'))
+//                         .map { t => LineItem(t._1, t._6) }
+//
+//    // filter the orders input
+//    val filteredOrders = orders filter { o => o.status == status && o.year > minYear && o.orderPriority.startsWith(priority) }
+//
+//    // join the filteres result with the lineitem input
+//    val prioritizedItems = filteredOrders join lineItems where { _.orderId } isEqualTo { _.orderId } map { (o, li) => PrioritizedOrder(o.orderId, o.shipPriority, li.extendedPrice) }
+//
+//    // group by and sum the joined data
+//    val prioritizedOrders = prioritizedItems groupBy { pi => (pi.orderId, pi.shipPriority) } reduce { (po1, po2) => po1.copy(revenue = po1.revenue + po2.revenue) }
+//
+//    // write the result as csv
+//    val output = prioritizedOrders.write(ordersOutput, CsvOutputFormat("\n", "|"))
+//
+//    val plan = new ScalaPlan(Seq(output), "Relational Query")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//
+//  override def getDescription() = {
+//    "Parameters: <orders>, <lineitem>, <output>, <degree-of-parallelism>"
+//  }
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(3).toInt, args(0), args(1), args(2))
+//  }
+//}
+//
+//
+///**
+// * Entry point to make the example standalone runnable with the local executor
+// */
+//object RunRelationalQuery {
+//
+//  def main(args: Array[String]) {
+//    val query = new RelationalQuery
+//
+//    if (args.size < 4) {
+//      println(query.getDescription)
+//      return
+//    }
+//    val plan = query.getScalaPlan(args(3).toInt, args(0), args(1), args(2))
+//    LocalExecutor.execute(plan)
+//  }
+//}
+//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
index 34e6d1e..e1ad641 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
@@ -1,150 +1,150 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.relational;
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-/**
- * Implements the following relational OLAP query as PACT program:
- * 
- * <code><pre>
- * SELECT r.pageURL, r.pageRank, r.avgDuration
- * FROM Documents d JOIN Rankings r
- * 	ON d.url = r.url
- * WHERE CONTAINS(d.text, [keywords])
- * 	AND r.rank > [rank]
- * 	AND NOT EXISTS (
- * 		SELECT * FROM Visits v
- * 		WHERE v.destUrl = d.url
- * 			AND v.visitDate < [date]); 
- *  * </pre></code> 
- * 
- * Table Schemas: <code><pre>
- * CREATE TABLE Documents (
- * 					url VARCHAR(100) PRIMARY KEY,
- * 					contents TEXT );
- * 
- * CREATE TABLE Rankings (
- * 					pageRank INT,
- * 					pageURL VARCHAR(100) PRIMARY KEY,     
- * 					avgDuration INT );       
- * 
- * CREATE TABLE Visits (
- * 					sourceIP VARCHAR(16),
- * 					destURL VARCHAR(100),
- * 					visitDate DATE,
- * 					adRevenue FLOAT,
- * 					userAgent VARCHAR(64),
- * 					countryCode VARCHAR(3),
- * 					languageCode VARCHAR(6),
- * 					searchWord VARCHAR(32),
- * 					duration INT );
- * </pre></code>
- * 
- */
-class WebLogAnalysis extends Program with ProgramDescription with Serializable {
-  
-  override def getDescription() = {
-    "Parameters: [numSubStasks], [docs], [rankings], [visits], [output]"
-  }
-  
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4))
-  }
-  
-  // document tuple
-  case class Doc(url: String, text: String)
-  
-  // rank tuple
-  case class Rank(rank: Int, url: String, avgDuration: Int)
-  
-  // visit tuple
-  case class Visit(url: String, date: String)
-  
-  // format rank for output
-  def formatRank = (r: Rank) => "%d|%s|%d".format(r.rank, r.url, r.avgDuration)
-
-  def getScalaPlan(numSubTasks: Int, docsInput: String, rankingsInput: String, visitsInput: String, ranksOutput: String) = {
-    
-    // read documents data
-    val docs = DataSource(docsInput, CsvInputFormat[Doc]("\n", '|'))
-    // read ranks data
-    val ranks = DataSource(rankingsInput, CsvInputFormat[Rank]("\n", '|'))
-    // read visits data and project to visits tuple afterwards
-    val visits = DataSource(visitsInput, CsvInputFormat[(String, String, String)]("\n", '|')) map (x => Visit(x._2, x._3))
-
-    // filter on documents that contain certain key words and project to URL
-    val filteredDocs = docs filter {d => d.text.contains(" editors ") && d.text.contains(" oscillations ") && d.text.contains(" convection ")} map { d => d.url }
-    filteredDocs.filterFactor(.15f)
-    filteredDocs.observes(d => (d.text))
-    filteredDocs.preserves({d => d.url}, {url => url} )
-    
-    // filter on ranks that have a certain minimum rank
-    val filteredRanks = ranks filter {r => r.rank > 50}
-    filteredRanks.filterFactor(.25f)
-    filteredRanks.observes(r => (r.rank))
-    filteredRanks.preserves({r => r}, {r => r} )
-    
-    // filter on visits of the year 2010 and project to URL
-    val filteredVisits = visits filter {v => v.date.substring(0,4).equals("2010")} map { v => v.url }
-    filteredVisits.filterFactor(.2f)
-    filteredVisits.observes(v => (v.date))
-    filteredVisits.preserves( {v => v.url}, {url => url} )
-    
-    // filter for ranks on documents that contain certain key words 
-    val ranksFilteredByDocs = filteredDocs join filteredRanks where {url => url} isEqualTo {r => r.url} map ((d,r) => r)
-    ranksFilteredByDocs.left.neglects( {d => d} )
-    ranksFilteredByDocs.right.preserves( {r => r}, {r => r} )
-    
-    // filter for ranks on documents that have not been visited in 2010
-    val ranksFilteredByDocsAndVisits = ranksFilteredByDocs cogroup filteredVisits where {r => r.url} isEqualTo {url => url} map ( (rs, vs) => if (vs.hasNext) Nil else rs.toList ) flatMap {rs => rs.iterator }
-    ranksFilteredByDocs.left.preserves( {r => r}, {r => r} )
-    ranksFilteredByDocs.right.neglects( {v => v} )
-    
-    // emit the resulting ranks
-    val output = ranksFilteredByDocsAndVisits.write(ranksOutput, DelimitedOutputFormat(formatRank))
-
-    val plan = new ScalaPlan(Seq(output), "WebLog Analysis")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-}
-
-
-/**
- * Entry point to make the example standalone runnable with the local executor
- */
-object RunWebLogAnalysis {
-  def main(args: Array[String]) {
-    val webLogAnalysis = new WebLogAnalysis
-    if (args.size < 5) {
-      println(webLogAnalysis.getDescription)
-      return
-    }
-    val plan = webLogAnalysis.getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4))
-    LocalExecutor.execute(plan)
-  }
-}
-
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.relational;
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+///**
+// * Implements the following relational OLAP query as PACT program:
+// *
+// * <code><pre>
+// * SELECT r.pageURL, r.pageRank, r.avgDuration
+// * FROM Documents d JOIN Rankings r
+// * 	ON d.url = r.url
+// * WHERE CONTAINS(d.text, [keywords])
+// * 	AND r.rank > [rank]
+// * 	AND NOT EXISTS (
+// * 		SELECT * FROM Visits v
+// * 		WHERE v.destUrl = d.url
+// * 			AND v.visitDate < [date]);
+// *  * </pre></code>
+// *
+// * Table Schemas: <code><pre>
+// * CREATE TABLE Documents (
+// * 					url VARCHAR(100) PRIMARY KEY,
+// * 					contents TEXT );
+// *
+// * CREATE TABLE Rankings (
+// * 					pageRank INT,
+// * 					pageURL VARCHAR(100) PRIMARY KEY,
+// * 					avgDuration INT );
+// *
+// * CREATE TABLE Visits (
+// * 					sourceIP VARCHAR(16),
+// * 					destURL VARCHAR(100),
+// * 					visitDate DATE,
+// * 					adRevenue FLOAT,
+// * 					userAgent VARCHAR(64),
+// * 					countryCode VARCHAR(3),
+// * 					languageCode VARCHAR(6),
+// * 					searchWord VARCHAR(32),
+// * 					duration INT );
+// * </pre></code>
+// *
+// */
+//class WebLogAnalysis extends Program with ProgramDescription with Serializable {
+//
+//  override def getDescription() = {
+//    "Parameters: [numSubStasks], [docs], [rankings], [visits], [output]"
+//  }
+//
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4))
+//  }
+//
+//  // document tuple
+//  case class Doc(url: String, text: String)
+//
+//  // rank tuple
+//  case class Rank(rank: Int, url: String, avgDuration: Int)
+//
+//  // visit tuple
+//  case class Visit(url: String, date: String)
+//
+//  // format rank for output
+//  def formatRank = (r: Rank) => "%d|%s|%d".format(r.rank, r.url, r.avgDuration)
+//
+//  def getScalaPlan(numSubTasks: Int, docsInput: String, rankingsInput: String, visitsInput: String, ranksOutput: String) = {
+//
+//    // read documents data
+//    val docs = DataSource(docsInput, CsvInputFormat[Doc]("\n", '|'))
+//    // read ranks data
+//    val ranks = DataSource(rankingsInput, CsvInputFormat[Rank]("\n", '|'))
+//    // read visits data and project to visits tuple afterwards
+//    val visits = DataSource(visitsInput, CsvInputFormat[(String, String, String)]("\n", '|')) map (x => Visit(x._2, x._3))
+//
+//    // filter on documents that contain certain key words and project to URL
+//    val filteredDocs = docs filter {d => d.text.contains(" editors ") && d.text.contains(" oscillations ") && d.text.contains(" convection ")} map { d => d.url }
+//    filteredDocs.filterFactor(.15f)
+//    filteredDocs.observes(d => (d.text))
+//    filteredDocs.preserves({d => d.url}, {url => url} )
+//
+//    // filter on ranks that have a certain minimum rank
+//    val filteredRanks = ranks filter {r => r.rank > 50}
+//    filteredRanks.filterFactor(.25f)
+//    filteredRanks.observes(r => (r.rank))
+//    filteredRanks.preserves({r => r}, {r => r} )
+//
+//    // filter on visits of the year 2010 and project to URL
+//    val filteredVisits = visits filter {v => v.date.substring(0,4).equals("2010")} map { v => v.url }
+//    filteredVisits.filterFactor(.2f)
+//    filteredVisits.observes(v => (v.date))
+//    filteredVisits.preserves( {v => v.url}, {url => url} )
+//
+//    // filter for ranks on documents that contain certain key words
+//    val ranksFilteredByDocs = filteredDocs join filteredRanks where {url => url} isEqualTo {r => r.url} map ((d,r) => r)
+//    ranksFilteredByDocs.left.neglects( {d => d} )
+//    ranksFilteredByDocs.right.preserves( {r => r}, {r => r} )
+//
+//    // filter for ranks on documents that have not been visited in 2010
+//    val ranksFilteredByDocsAndVisits = ranksFilteredByDocs cogroup filteredVisits where {r => r.url} isEqualTo {url => url} map ( (rs, vs) => if (vs.hasNext) Nil else rs.toList ) flatMap {rs => rs.iterator }
+//    ranksFilteredByDocs.left.preserves( {r => r}, {r => r} )
+//    ranksFilteredByDocs.right.neglects( {v => v} )
+//
+//    // emit the resulting ranks
+//    val output = ranksFilteredByDocsAndVisits.write(ranksOutput, DelimitedOutputFormat(formatRank))
+//
+//    val plan = new ScalaPlan(Seq(output), "WebLog Analysis")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//}
+//
+//
+///**
+// * Entry point to make the example standalone runnable with the local executor
+// */
+//object RunWebLogAnalysis {
+//  def main(args: Array[String]) {
+//    val webLogAnalysis = new WebLogAnalysis
+//    if (args.size < 5) {
+//      println(webLogAnalysis.getDescription)
+//      return
+//    }
+//    val plan = webLogAnalysis.getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4))
+//    LocalExecutor.execute(plan)
+//  }
+//}
+//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
index 81a28a4..52bfc15 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
@@ -1,105 +1,105 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.testing
-
-// Uncomment if you need to rebuild it for PackagedProgramEndToEndTest
-//
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//
-//class KMeansForTest extends Program with ProgramDescription {
-//
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4).toInt)
-//  }
-//
-//  case class Point(x: Double, y: Double, z: Double) {
-//    def computeEuclidianDistance(other: Point) = other match {
-//      case Point(x2, y2, z2) => math.sqrt(math.pow(x - x2, 2) + math.pow(y - y2, 2) + math.pow(z - z2, 2))
-//    }
-//  }
-//
-//  case class Distance(dataPoint: Point, clusterId: Int, distance: Double)
-//
-//  def asPointSum = (pid: Int, dist: Distance) => dist.clusterId -> PointSum(1, dist.dataPoint)
-//
-//  //  def sumPointSums = (dataPoints: Iterator[(Int, PointSum)]) => dataPoints.reduce { (z, v) => z.copy(_2 = z._2 + v._2) }
-//  def sumPointSums = (dataPoints: Iterator[(Int, PointSum)]) => {
-//    dataPoints.reduce { (z, v) => z.copy(_2 = z._2 + v._2) }
-//  }
-//
-//
-//  case class PointSum(count: Int, pointSum: Point) {
-//    def +(that: PointSum) = that match {
-//      case PointSum(c, Point(x, y, z)) => PointSum(count + c, Point(x + pointSum.x, y + pointSum.y, z + pointSum.z))
-//    }
-//
-//    def toPoint() = Point(round(pointSum.x / count), round(pointSum.y / count), round(pointSum.z / count))
-//
-//    // Rounding ensures that we get the same results in a multi-iteration run
-//    // as we do in successive single-iteration runs, since the output format
-//    // only contains two decimal places.
-//    private def round(d: Double) = math.round(d * 100.0) / 100.0;
-//  }
-//
-//  def parseInput = (line: String) => {
-//    val PointInputPattern = """(\d+)\|-?(\d+\.\d+)\|-?(\d+\.\d+)\|-?(\d+\.\d+)\|""".r
-//    val PointInputPattern(id, x, y, z) = line
-//    (id.toInt, Point(x.toDouble, y.toDouble, z.toDouble))
-//  }
-//
-//  def formatOutput = (cid: Int, p: Point) => "%d|%.2f|%.2f|%.2f|".format(cid, p.x, p.y, p.z)
-//
-//  def computeDistance(p: (Int, Point), c: (Int, Point)) = {
-//    val ((pid, dataPoint), (cid, clusterPoint)) = (p, c)
-//    val distToCluster = dataPoint.computeEuclidianDistance(clusterPoint)
-//
-//    pid -> Distance(dataPoint, cid, distToCluster)
-//  }
-//
-//
-//  def getScalaPlan(numSubTasks: Int, dataPointInput: String, clusterInput: String, clusterOutput: String, numIterations: Int) = {
-//    val dataPoints = DataSource(dataPointInput, DelimitedInputFormat(parseInput))
-//    val clusterPoints = DataSource(clusterInput, DelimitedInputFormat(parseInput))
-//
-//    val finalCenters = clusterPoints.iterate(numIterations, { centers =>
-//
-//      val distances = dataPoints cross centers map computeDistance
-//      val nearestCenters = distances groupBy { case (pid, _) => pid } reduceGroup { ds => ds.minBy(_._2.distance) } map asPointSum.tupled
-//      val newCenters = nearestCenters groupBy { case (cid, _) => cid } reduceGroup sumPointSums map { case (cid, pSum) => cid -> pSum.toPoint() }
-//
-//      newCenters
-//    })
-//
-//    val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatOutput.tupled))
-//
-//    val plan = new ScalaPlan(Seq(output), "KMeans Iteration (ONLY FOR TESTING)")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//
-//  override def getDescription() = {
-//    "Parameters: [numSubStasksS] [dataPoints] [clusterCenters] [output] [numIterations]"
-//  }
-//}
\ No newline at end of file
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.testing
+//
+//// Uncomment if you need to rebuild it for PackagedProgramEndToEndTest
+////
+////import org.apache.flink.api.common.Program
+////import org.apache.flink.api.common.ProgramDescription
+////
+////import org.apache.flink.api.scala._
+////import org.apache.flink.api.scala.operators._
+////
+////
+////class KMeansForTest extends Program with ProgramDescription {
+////
+////  override def getPlan(args: String*) = {
+////    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4).toInt)
+////  }
+////
+////  case class Point(x: Double, y: Double, z: Double) {
+////    def computeEuclidianDistance(other: Point) = other match {
+////      case Point(x2, y2, z2) => math.sqrt(math.pow(x - x2, 2) + math.pow(y - y2, 2) + math.pow(z - z2, 2))
+////    }
+////  }
+////
+////  case class Distance(dataPoint: Point, clusterId: Int, distance: Double)
+////
+////  def asPointSum = (pid: Int, dist: Distance) => dist.clusterId -> PointSum(1, dist.dataPoint)
+////
+////  //  def sumPointSums = (dataPoints: Iterator[(Int, PointSum)]) => dataPoints.reduce { (z, v) => z.copy(_2 = z._2 + v._2) }
+////  def sumPointSums = (dataPoints: Iterator[(Int, PointSum)]) => {
+////    dataPoints.reduce { (z, v) => z.copy(_2 = z._2 + v._2) }
+////  }
+////
+////
+////  case class PointSum(count: Int, pointSum: Point) {
+////    def +(that: PointSum) = that match {
+////      case PointSum(c, Point(x, y, z)) => PointSum(count + c, Point(x + pointSum.x, y + pointSum.y, z + pointSum.z))
+////    }
+////
+////    def toPoint() = Point(round(pointSum.x / count), round(pointSum.y / count), round(pointSum.z / count))
+////
+////    // Rounding ensures that we get the same results in a multi-iteration run
+////    // as we do in successive single-iteration runs, since the output format
+////    // only contains two decimal places.
+////    private def round(d: Double) = math.round(d * 100.0) / 100.0;
+////  }
+////
+////  def parseInput = (line: String) => {
+////    val PointInputPattern = """(\d+)\|-?(\d+\.\d+)\|-?(\d+\.\d+)\|-?(\d+\.\d+)\|""".r
+////    val PointInputPattern(id, x, y, z) = line
+////    (id.toInt, Point(x.toDouble, y.toDouble, z.toDouble))
+////  }
+////
+////  def formatOutput = (cid: Int, p: Point) => "%d|%.2f|%.2f|%.2f|".format(cid, p.x, p.y, p.z)
+////
+////  def computeDistance(p: (Int, Point), c: (Int, Point)) = {
+////    val ((pid, dataPoint), (cid, clusterPoint)) = (p, c)
+////    val distToCluster = dataPoint.computeEuclidianDistance(clusterPoint)
+////
+////    pid -> Distance(dataPoint, cid, distToCluster)
+////  }
+////
+////
+////  def getScalaPlan(numSubTasks: Int, dataPointInput: String, clusterInput: String, clusterOutput: String, numIterations: Int) = {
+////    val dataPoints = DataSource(dataPointInput, DelimitedInputFormat(parseInput))
+////    val clusterPoints = DataSource(clusterInput, DelimitedInputFormat(parseInput))
+////
+////    val finalCenters = clusterPoints.iterate(numIterations, { centers =>
+////
+////      val distances = dataPoints cross centers map computeDistance
+////      val nearestCenters = distances groupBy { case (pid, _) => pid } reduceGroup { ds => ds.minBy(_._2.distance) } map asPointSum.tupled
+////      val newCenters = nearestCenters groupBy { case (cid, _) => cid } reduceGroup sumPointSums map { case (cid, pSum) => cid -> pSum.toPoint() }
+////
+////      newCenters
+////    })
+////
+////    val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatOutput.tupled))
+////
+////    val plan = new ScalaPlan(Seq(output), "KMeans Iteration (ONLY FOR TESTING)")
+////    plan.setDefaultParallelism(numSubTasks)
+////    plan
+////  }
+////
+////  override def getDescription() = {
+////    "Parameters: [numSubStasksS] [dataPoints] [clusterCenters] [output] [numIterations]"
+////  }
+////}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
index 3b2b2b4..22838e5 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,59 +15,85 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.flink.examples.scala.wordcount
 
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
 import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
+import org.apache.flink.example.java.wordcount.util.WordCountData
 
 /**
- * Implementation of word count in Scala.
+ * Implements the "WordCount" program that computes a simple word occurrence histogram
+ * over text files. 
+ *
+ * The input is a plain text file with lines separated by newline characters.
+ *
+ * Usage:
+ * {{{
+ *   WordCount <text path> <result path>>
+ * }}}
+ *
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.example.java.wordcount.util.WordCountData]]
+ *
+ * This example shows how to:
+ *
+ *   - write a simple Flink program.
+ *   - use Tuple data types.
+ *   - write and use user-defined functions.
+ *
  */
-class WordCount extends Program with ProgramDescription with Serializable {
+object WordCount {
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
 
-  def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
-    
-    val input = TextFile(textInput)
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val text = getTextDataSet(env)
 
-    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } map { (_, 1) } }
-    val counts = words groupBy { case (word, _) => word } reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
+    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
+      .map { (_, 1) }
+      .groupBy(0)
+      .sum(1)
 
-    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
-  
-    val plan = new ScalaPlan(Seq(output), "Word Count")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-  
-  override def getDescription() = {
-    "Parameters: <numSubStasks> <input> <output>"
+    if (fileOutput) {
+      counts.writeAsCsv(outputPath, "\n", " ")
+    } else {
+      counts.print()
+    }
+
+    env.execute("Scala WordCount Example")
   }
-  override def getPlan(args: String*) = {
-    if (args.size < 3) {
-      println(getDescription)
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      fileOutput = true
+      if (args.length == 2) {
+        textPath = args(0)
+        outputPath = args(1)
+      } else {
+        System.err.println("Usage: WordCount <text path> <result path>")
+        false
+      }
+    } else {
+      System.out.println("Executing WordCount example with built-in default data.")
+      System.out.println("  Provide parameters to read input data from a file.")
+      System.out.println("  Usage: WordCount <text path> <result path>")
     }
-    getScalaPlan(args(0).toInt, args(1), args(2))
+    true
   }
-}
 
-/**
- * Entry point to make the example standalone runnable with the local executor
- */
-object RunWordCount {
-  def main(args: Array[String]) {
-    val wc = new WordCount
-    if (args.size < 3) {
-      println(wc.getDescription)
-      return
+  private def getTextDataSet(env: ExecutionEnvironment): DataSet[String] = {
+    if (fileOutput) {
+      env.readTextFile(textPath)
+    }
+    else {
+      env.fromCollection(WordCountData.WORDS)
     }
-    val plan = wc.getScalaPlan(args(0).toInt, args(1), args(2))
-    LocalExecutor.execute(plan)
   }
+
+  private var fileOutput: Boolean = false
+  private var textPath: String = null
+  private var outputPath: String = null
 }
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
index 1268cd1..f71b18d 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
@@ -1,63 +1,63 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.wordcount
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-
-/**
- * Implementation of word count in Scala. This example uses the built in count function for tuples.
- */
-class WordCountWithCount extends WordCount {
-
-  override def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
-    val input = TextFile(textInput)
-
-    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
-    val counts = words groupBy { x => x } count()
-
-    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
-  
-    val plan = new ScalaPlan(Seq(output), "Word Count")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-}
-
-
-/**
- * Entry point to make the example standalone runnable with the local executor.
- */
-object RunWordCountWithCount {
-  def main(args: Array[String]) {
-    val wc = new WordCountWithCount
-    if (args.size < 3) {
-      println(wc.getDescription)
-      return
-    }
-    val plan = wc.getScalaPlan(args(0).toInt, args(1), args(2))
-    LocalExecutor.execute(plan)
-  }
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.wordcount
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//
+///**
+// * Implementation of word count in Scala. This example uses the built in count function for tuples.
+// */
+//class WordCountWithCount extends WordCount {
+//
+//  override def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
+//    val input = TextFile(textInput)
+//
+//    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
+//    val counts = words groupBy { x => x } count()
+//
+//    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
+//
+//    val plan = new ScalaPlan(Seq(output), "Word Count")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//}
+//
+//
+///**
+// * Entry point to make the example standalone runnable with the local executor.
+// */
+//object RunWordCountWithCount {
+//  def main(args: Array[String]) {
+//    val wc = new WordCountWithCount
+//    if (args.size < 3) {
+//      println(wc.getDescription)
+//      return
+//    }
+//    val plan = wc.getScalaPlan(args(0).toInt, args(1), args(2))
+//    LocalExecutor.execute(plan)
+//  }
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
index 00489a6..2ee0c43 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
@@ -1,59 +1,59 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.examples.scala.wordcount
-
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-
-import org.apache.flink.types.IntValue
-import org.apache.flink.types.StringValue
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-
-
-/**
- * Implementation of word count in Scala, using a user defined type rather than one of the
- * built-in supported types like primitives, tuples, or other (nested) case classes.
- */
-class WordCountWithUserDefinedType extends Program with Serializable {
-
-  def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
-    val input = TextFile(textInput)
-
-    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } map { w => (new StringValue(w), new IntValue(1)) } }
-    
-    val counts = words
-      .groupBy { case (word, _) => word }
-      .reduce { (w1, w2) => (w1._1, new IntValue(w1._2.getValue + w2._2.getValue)) }
-
-    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
-  
-    val plan = new ScalaPlan(Seq(output), "Word Count (immutable)")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
-  }
-  
-
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2))
-  }
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//
+//package org.apache.flink.examples.scala.wordcount
+//
+//import org.apache.flink.client.LocalExecutor
+//import org.apache.flink.api.common.Program
+//import org.apache.flink.api.common.ProgramDescription
+//
+//import org.apache.flink.types.IntValue
+//import org.apache.flink.types.StringValue
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//
+//
+///**
+// * Implementation of word count in Scala, using a user defined type rather than one of the
+// * built-in supported types like primitives, tuples, or other (nested) case classes.
+// */
+//class WordCountWithUserDefinedType extends Program with Serializable {
+//
+//  def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
+//    val input = TextFile(textInput)
+//
+//    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } map { w => (new StringValue(w), new IntValue(1)) } }
+//
+//    val counts = words
+//      .groupBy { case (word, _) => word }
+//      .reduce { (w1, w2) => (w1._1, new IntValue(w1._2.getValue + w2._2.getValue)) }
+//
+//    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
+//
+//    val plan = new ScalaPlan(Seq(output), "Word Count (immutable)")
+//    plan.setDefaultParallelism(numSubTasks)
+//    plan
+//  }
+//
+//
+//  override def getPlan(args: String*) = {
+//    getScalaPlan(args(0).toInt, args(1), args(2))
+//  }
+//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java/pom.xml b/flink-java/pom.xml
index 4a32f10..5165a78 100644
--- a/flink-java/pom.xml
+++ b/flink-java/pom.xml
@@ -58,4 +58,21 @@ under the License.
 			<version>2.24.0</version>
 		</dependency>
 	</dependencies>
+
+    <!-- Because flink-scala uses it in tests -->
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
index be5fe48..21efda2 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
@@ -36,7 +36,7 @@ import org.apache.flink.api.java.aggregation.Aggregations;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable;
 import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.Collector;
 
@@ -67,7 +67,7 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 			throw new InvalidProgramException("Aggregating on field positions is only possible on tuple data types.");
 		}
 		
-		TupleTypeInfo<?> inType = (TupleTypeInfo<?>) input.getType();
+		TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) input.getType();
 		
 		if (field < 0 || field >= inType.getArity()) {
 			throw new IllegalArgumentException("Aggregation field position is out of range.");
@@ -99,7 +99,7 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 			throw new InvalidProgramException("Aggregating on field positions is only possible on tuple data types.");
 		}
 		
-		TupleTypeInfo<?> inType = (TupleTypeInfo<?>) input.getDataSet().getType();
+		TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) input.getDataSet().getType();
 		
 		if (field < 0 || field >= inType.getArity()) {
 			throw new IllegalArgumentException("Aggregation field position is out of range.");
@@ -118,7 +118,7 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 	public AggregateOperator<IN> and(Aggregations function, int field) {
 		Validate.notNull(function);
 		
-		TupleTypeInfo<?> inType = (TupleTypeInfo<?>) getType();
+		TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) getType();
 		
 		if (field < 0 || field >= inType.getArity()) {
 			throw new IllegalArgumentException("Aggregation field position is out of range.");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
index a24a093..ac04d4f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
@@ -51,7 +51,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 
 	private final CrossFunction<I1, I2, OUT> function;
 
-	protected CrossOperator(DataSet<I1> input1, DataSet<I2> input2,
+	public CrossOperator(DataSet<I1> input1, DataSet<I2> input2,
 							CrossFunction<I1, I2, OUT> function,
 							TypeInformation<OUT> returnType)
 	{

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
index 200c4de..66bcb92 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
@@ -55,11 +55,11 @@ public abstract class Grouping<T> {
 	}
 	
 	
-	protected DataSet<T> getDataSet() {
+	public DataSet<T> getDataSet() {
 		return this.dataSet;
 	}
 	
-	protected Keys<T> getKeys() {
+	public Keys<T> getKeys() {
 		return this.keys;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index f242453..4cedb50 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -165,7 +165,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		@SuppressWarnings("unused")
 		private boolean preserve2;
 		
-		protected EquiJoin(DataSet<I1> input1, DataSet<I2> input2, 
+		public EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
 				Keys<I1> keys1, Keys<I2> keys2, FlatJoinFunction<I1, I2, OUT> function,
 				TypeInformation<OUT> returnType, JoinHint hint)
 		{
@@ -184,7 +184,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			}
 		}
 
-		protected EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
+		public EquiJoin(DataSet<I1> input1, DataSet<I2> input2,
 				Keys<I1> keys1, Keys<I2> keys2, FlatJoinFunction<I1, I2, OUT> generatedFunction, JoinFunction<I1, I2, OUT> function,
 				TypeInformation<OUT> returnType, JoinHint hint)
 		{
@@ -520,11 +520,11 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			return new EquiJoin<I1, I2, R>(getInput1(), getInput2(), getKeys1(), getKeys2(), generatedFunction, function, returnType, getJoinHint());
 		}
 
-		private static class WrappingFlatJoinFunction<IN1, IN2, OUT> extends WrappingFunction<JoinFunction<IN1,IN2,OUT>> implements FlatJoinFunction<IN1, IN2, OUT> {
+		public static class WrappingFlatJoinFunction<IN1, IN2, OUT> extends WrappingFunction<JoinFunction<IN1,IN2,OUT>> implements FlatJoinFunction<IN1, IN2, OUT> {
 
 			private static final long serialVersionUID = 1L;
 
-			private WrappingFlatJoinFunction(JoinFunction<IN1, IN2, OUT> wrappedFunction) {
+			public WrappingFlatJoinFunction(JoinFunction<IN1, IN2, OUT> wrappedFunction) {
 				super(wrappedFunction);
 			}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
index 125ad85..6325db3 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
@@ -55,7 +55,8 @@ public abstract class Keys<T> {
 
 		public FieldPositionKeys(int[] groupingFields, TypeInformation<T> type, boolean allowEmpty) {
 			if (!type.isTupleType()) {
-				throw new InvalidProgramException("Specifying keys via field positions is only valid for tuple data types");
+				throw new InvalidProgramException("Specifying keys via field positions is only valid" +
+						"for tuple data types. Type: " + type);
 			}
 
 			if (!allowEmpty && (groupingFields == null || groupingFields.length == 0)) {


[03/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/JoinOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/JoinOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/JoinOperator.scala
deleted file mode 100644
index a92d37a..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/JoinOperator.scala
+++ /dev/null
@@ -1,296 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import java.util.{ Iterator => JIterator }
-
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.java.record.operators.JoinOperator
-import org.apache.flink.api.java.record.functions.{JoinFunction => JJoinFunction}
-import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper
-import org.apache.flink.configuration.Configuration
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.codegen.{MacroContextHolder, Util}
-import org.apache.flink.api.scala.functions.{JoinFunctionBase, JoinFunction, FlatJoinFunction}
-import org.apache.flink.api.scala.analysis.FieldSelector
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.TwoInputHintable
-
-class JoinDataSet[LeftIn, RightIn](val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def where[Key](keyFun: LeftIn => Key) = macro JoinMacros.whereImpl[LeftIn, RightIn, Key]
-}
-
-class JoinDataSetWithWhere[LeftIn, RightIn, Key](val leftKey: List[Int], val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def isEqualTo[Key](keyFun: RightIn => Key) = macro JoinMacros.isEqualToImpl[LeftIn, RightIn, Key]
-}
-
-class JoinDataSetWithWhereAndEqual[LeftIn, RightIn](val leftKey: List[Int], val rightKey: List[Int], val leftInput: DataSet[LeftIn], val rightInput: DataSet[RightIn]) {
-  def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro JoinMacros.map[LeftIn, RightIn, Out]
-  def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out] = macro JoinMacros.flatMap[LeftIn, RightIn, Out]
-  def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)] = macro JoinMacros.filter[LeftIn, RightIn]
-}
-
-class NoKeyMatchBuilder(s: JJoinFunction) extends JoinOperator.Builder(new UserCodeObjectWrapper(s))
-
-object JoinMacros {
-  
-  def whereImpl[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Key: c.WeakTypeTag](c: Context { type PrefixType = JoinDataSet[LeftIn, RightIn] })(keyFun: c.Expr[LeftIn => Key]): c.Expr[JoinDataSetWithWhere[LeftIn, RightIn, Key]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val keySelection = slave.getSelector(keyFun)
-
-    val helper = reify {
-      val helper = c.prefix.splice
-      new JoinDataSetWithWhere[LeftIn, RightIn, Key](keySelection.splice, helper.leftInput, helper.rightInput)
-    }
-
-    return helper
-  }
-  
-  def isEqualToImpl[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Key: c.WeakTypeTag](c: Context { type PrefixType = JoinDataSetWithWhere[LeftIn, RightIn, Key] })(keyFun: c.Expr[RightIn => Key]): c.Expr[JoinDataSetWithWhereAndEqual[LeftIn, RightIn]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val keySelection = slave.getSelector(keyFun)
-
-    val helper = reify {
-      val helper = c.prefix.splice
-      new JoinDataSetWithWhereAndEqual[LeftIn, RightIn](helper.leftKey, keySelection.splice, helper.leftInput, helper.rightInput)
-    }
-
-    return helper
-  }
-
-  def map[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = JoinDataSetWithWhereAndEqual[LeftIn, RightIn] })(fun: c.Expr[(LeftIn, RightIn) => Out]): c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-
-    val stub: c.Expr[JoinFunctionBase[LeftIn, RightIn, Out]] = if (fun.actualType <:< weakTypeOf[JoinFunction[LeftIn, RightIn, Out]])
-      reify { fun.splice.asInstanceOf[JoinFunctionBase[LeftIn, RightIn, Out]] }
-    else reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-      new JoinFunctionBase[LeftIn, RightIn, Out] {
-        override def join(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = {
-          val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-          val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-          val output = fun.splice.apply(left, right)
-
-          leftRecord.setNumFields(outputLength)
-          for (field <- leftDiscard)
-            leftRecord.setNull(field)
-
-          leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo)
-          leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo)
-
-          serializer.serialize(output, leftRecord)
-          out.collect(leftRecord)
-        }
-      }
-    }
-    val contract = reify {
-      val helper: JoinDataSetWithWhereAndEqual[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val leftKeySelector = new FieldSelector(generatedStub.leftInputUDT, helper.leftKey)
-      val rightKeySelector = new FieldSelector(generatedStub.rightInputUDT, helper.rightKey)
-
-      val builder = new NoKeyMatchBuilder(generatedStub).input1(leftInput).input2(rightInput)
-
-      val leftKeyPositions = leftKeySelector.selectedFields.toIndexArray
-      val rightKeyPositions = rightKeySelector.selectedFields.toIndexArray
-
-      val keyTypes = generatedStub.leftInputUDT.getKeySet(leftKeyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), leftKeyPositions(i), rightKeyPositions(i)) }
-
-      
-      
-      val ret = new JoinOperator(builder) with TwoInputKeyedScalaOperator[LeftIn, RightIn, Out] {
-        override val leftKey: FieldSelector = leftKeySelector
-        override val rightKey: FieldSelector = rightKeySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-      }
-      new DataSet[Out](ret) with TwoInputHintable[LeftIn, RightIn, Out] {}
-    }
-    
-    val result = c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-  def flatMap[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = JoinDataSetWithWhereAndEqual[LeftIn, RightIn] })(fun: c.Expr[(LeftIn, RightIn) => Iterator[Out]]): c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-
-    val stub: c.Expr[JoinFunctionBase[LeftIn, RightIn, Out]] = if (fun.actualType <:< weakTypeOf[JoinFunction[LeftIn, RightIn, Out]])
-      reify { fun.splice.asInstanceOf[JoinFunctionBase[LeftIn, RightIn, Out]] }
-    else reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-      new JoinFunctionBase[LeftIn, RightIn, Out] {
-        override def join(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = {
-          val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-          val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-          val output = fun.splice.apply(left, right)
-
-          if (output.nonEmpty) {
-
-            leftRecord.setNumFields(outputLength)
-
-            for (field <- leftDiscard)
-              leftRecord.setNull(field)
-
-            leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo)
-            leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo)
-
-            for (item <- output) {
-              serializer.serialize(item, leftRecord)
-              out.collect(leftRecord)
-            }
-          }
-        }
-      }
-    }
-    val contract = reify {
-      val helper: JoinDataSetWithWhereAndEqual[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val leftKeySelector = new FieldSelector(generatedStub.leftInputUDT, helper.leftKey)
-      val rightKeySelector = new FieldSelector(generatedStub.rightInputUDT, helper.rightKey)
-      val builder = new NoKeyMatchBuilder(generatedStub).input1(leftInput).input2(rightInput)
-
-      val leftKeyPositions = leftKeySelector.selectedFields.toIndexArray
-      val rightKeyPositions = rightKeySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.leftInputUDT.getKeySet(leftKeyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), leftKeyPositions(i), rightKeyPositions(i)) }
-      
-      
-      val ret = new JoinOperator(builder) with TwoInputKeyedScalaOperator[LeftIn, RightIn, Out] {
-        override val leftKey: FieldSelector = leftKeySelector
-        override val rightKey: FieldSelector = rightKeySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-      }
-      new DataSet[Out](ret) with TwoInputHintable[LeftIn, RightIn, Out] {}
-    }
-
-    val result = c.Expr[DataSet[Out] with TwoInputHintable[LeftIn, RightIn, Out]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-  def filter[LeftIn: c.WeakTypeTag, RightIn: c.WeakTypeTag](c: Context { type PrefixType = JoinDataSetWithWhereAndEqual[LeftIn, RightIn] })(fun: c.Expr[(LeftIn, RightIn) => Boolean]): c.Expr[DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val (udtLeftIn, createUdtLeftIn) = slave.mkUdtClass[LeftIn]
-    val (udtRightIn, createUdtRightIn) = slave.mkUdtClass[RightIn]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[(LeftIn, RightIn)]
-
-    val stub: c.Expr[JoinFunctionBase[LeftIn, RightIn, (LeftIn, RightIn)]] = if (fun.actualType <:< weakTypeOf[JoinFunction[LeftIn, RightIn, (LeftIn, RightIn)]])
-      reify { fun.splice.asInstanceOf[JoinFunctionBase[LeftIn, RightIn, (LeftIn, RightIn)]] }
-    else reify {
-      implicit val leftInputUDT: UDT[LeftIn] = c.Expr[UDT[LeftIn]](createUdtLeftIn).splice
-      implicit val rightInputUDT: UDT[RightIn] = c.Expr[UDT[RightIn]](createUdtRightIn).splice
-      implicit val outputUDT: UDT[(LeftIn, RightIn)] = c.Expr[UDT[(LeftIn, RightIn)]](createUdtOut).splice
-      new JoinFunctionBase[LeftIn, RightIn, (LeftIn, RightIn)] {
-        override def join(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = {
-          val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-          val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-          if (fun.splice.apply(left, right)) {
-            val output = (left, right)
-            leftRecord.setNumFields(outputLength)
-            serializer.serialize(output, leftRecord)
-            out.collect(leftRecord)
-          }
-        }
-      }
-    }
-    val contract = reify {
-      val helper: JoinDataSetWithWhereAndEqual[LeftIn, RightIn] = c.prefix.splice
-      val leftInput = helper.leftInput.contract
-      val rightInput = helper.rightInput.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val leftKeySelector = new FieldSelector(generatedStub.leftInputUDT, helper.leftKey)
-      val rightKeySelector = new FieldSelector(generatedStub.rightInputUDT, helper.rightKey)
-      val builder = new NoKeyMatchBuilder(generatedStub).input1(leftInput).input2(rightInput)
-
-      val leftKeyPositions = leftKeySelector.selectedFields.toIndexArray
-      val rightKeyPositions = rightKeySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.leftInputUDT.getKeySet(leftKeyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), leftKeyPositions(i), rightKeyPositions(i)) }
-      
-      
-      val ret = new JoinOperator(builder) with TwoInputKeyedScalaOperator[LeftIn, RightIn, (LeftIn, RightIn)] {
-        override val leftKey: FieldSelector = leftKeySelector
-        override val rightKey: FieldSelector = rightKeySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFieldsFirst(
-            Util.filterNonForwards(getUDF.getLeftForwardIndexArrayFrom, getUDF.getLeftForwardIndexArrayTo)),
-          Annotations.getConstantFieldsSecond(
-            Util.filterNonForwards(getUDF.getRightForwardIndexArrayFrom, getUDF.getRightForwardIndexArrayTo)))
-      }
-      new DataSet[(LeftIn, RightIn)](ret) with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)] {}
-    }
-
-    val result = c.Expr[DataSet[(LeftIn, RightIn)] with TwoInputHintable[LeftIn, RightIn, (LeftIn, RightIn)]](Block(List(udtLeftIn, udtRightIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/MapOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/MapOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/MapOperator.scala
deleted file mode 100644
index 824eb9d..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/MapOperator.scala
+++ /dev/null
@@ -1,218 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-
-import scala.reflect.macros.Context
-
-import org.apache.flink.api.scala.codegen.{MacroContextHolder, Util}
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.OneInputHintable
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.functions.{MapFunction, MapFunctionBase}
-import org.apache.flink.api.scala.functions.{FlatMapFunction, FilterFunction}
-
-import org.apache.flink.api.java.record.operators.MapOperator
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-
-
-object MapMacros {
-
-  def map[In: c.WeakTypeTag, Out: c.WeakTypeTag]
-      (c: Context { type PrefixType = DataSet[In] })
-      (fun: c.Expr[In => Out]): c.Expr[DataSet[Out]
-    with OneInputHintable[In, Out]] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-//    val (paramName, udfBody) = slave.extractOneInputUdf(fun.tree)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]()
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]()
-
-    val stub: c.Expr[MapFunctionBase[In, Out]] =
-      if (fun.actualType <:< weakTypeOf[MapFunction[In, Out]])
-        reify { fun.splice.asInstanceOf[MapFunctionBase[In, Out]] }
-      else
-        reify {
-          implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice
-          implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-          new MapFunctionBase[In, Out] {
-            // val userFun = ClosureCleaner.clean(fun.splice)
-            // val userFun = fun.splice
-            override def map(record: Record, out: Collector[Record]) = {
-              val input = deserializer.deserializeRecyclingOn(record)
-              val output = fun.splice.apply(input)
-
-              record.setNumFields(outputLength)
-
-              for (field <- discard)
-                record.setNull(field)
-
-              serializer.serialize(output, record)
-              out.collect(record)
-            }
-          }
-        }
-
-    val contract = reify {
-      val input = c.prefix.splice.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val builder = MapOperator.builder(generatedStub).input(input)
-      
-      val contract = new MapOperator(builder) with OneInputScalaOperator[In, Out] {
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFields(
-            Util.filterNonForwards(getUDF.getForwardIndexArrayFrom, getUDF.getForwardIndexArrayTo)))
-      }
-      val stream = new DataSet[Out](contract) with OneInputHintable[In, Out] {}
-      contract.persistHints = { () => stream.applyHints(contract) }
-      stream
-    }
-
-    val result = c.Expr[DataSet[Out]
-      with OneInputHintable[In, Out]](Block(List(udtIn, udtOut), contract.tree))
-
-    result
-  }
-  
-  def flatMap[In: c.WeakTypeTag, Out: c.WeakTypeTag]
-      (c: Context { type PrefixType = DataSet[In] })
-      (fun: c.Expr[In => Iterator[Out]]): c.Expr[DataSet[Out]
-    with OneInputHintable[In, Out]] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-//    val (paramName, udfBody) = slave.extractOneInputUdf(fun.tree)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]()
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]()
-
-    val stub: c.Expr[MapFunctionBase[In, Out]] =
-      if (fun.actualType <:< weakTypeOf[FlatMapFunction[In, Out]])
-        reify { fun.splice.asInstanceOf[MapFunctionBase[In, Out]] }
-      else reify {
-        implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice
-        implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-        new MapFunctionBase[In, Out] {
-          override def map(record: Record, out: Collector[Record]) = {
-            val input = deserializer.deserializeRecyclingOn(record)
-            val output = fun.splice.apply(input)
-
-            if (output.nonEmpty) {
-
-              record.setNumFields(outputLength)
-
-              for (field <- discard)
-                record.setNull(field)
-
-              for (item <- output) {
-
-                serializer.serialize(item, record)
-                out.collect(record)
-              }
-            }
-          }
-        }
-      }
-    val contract = reify {
-      val input = c.prefix.splice.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val builder = MapOperator.builder(generatedStub).input(input)
-      
-      val contract = new MapOperator(builder) with OneInputScalaOperator[In, Out] {
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFields(
-            Util.filterNonForwards(getUDF.getForwardIndexArrayFrom, getUDF.getForwardIndexArrayTo)))
-      }
-      val stream = new DataSet[Out](contract) with OneInputHintable[In, Out] {}
-      contract.persistHints = { () => stream.applyHints(contract) }
-      stream
-    }
-
-    val result = c.Expr[DataSet[Out] with OneInputHintable[In, Out]](Block(List(udtIn, udtOut),
-      contract.tree))
-
-    result
-  }
-  
-  def filter[In: c.WeakTypeTag]
-      (c: Context { type PrefixType = DataSet[In] })
-      (fun: c.Expr[In => Boolean]): c.Expr[DataSet[In]
-
-    with OneInputHintable[In, In]] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-
-    //    val (paramName, udfBody) = slave.extractOneInputUdf(fun.tree)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]()
-
-    val stub: c.Expr[MapFunctionBase[In, In]] =
-      if (fun.actualType <:< weakTypeOf[FilterFunction[In, In]])
-        reify { fun.splice.asInstanceOf[MapFunctionBase[In, In]] }
-      else
-        reify {
-          implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice
-          new MapFunctionBase[In, In] {
-            override def map(record: Record, out: Collector[Record]) = {
-              val input = deserializer.deserializeRecyclingOn(record)
-              if (fun.splice.apply(input)) {
-        	      out.collect(record)
-              }
-            }
-          }
-        }
-    val contract = reify {
-      val input = c.prefix.splice.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val builder = MapOperator.builder(generatedStub).input(input)
-      
-      val contract = new MapOperator(builder) with OneInputScalaOperator[In, In] {
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFields(
-            Util.filterNonForwards(getUDF.getForwardIndexArrayFrom, getUDF.getForwardIndexArrayTo)))
-      }
-      val stream = new DataSet[In](contract) with OneInputHintable[In, In] {}
-      contract.persistHints = { () =>
-        stream.applyHints(contract)
-        0 until generatedStub.udf.getOutputLength foreach { i => stream.markCopied(i, i) }
-      }
-      stream
-    }
-
-    val result = c.Expr[DataSet[In]
-      with OneInputHintable[In, In]](Block(List(udtIn), contract.tree))
-
-    result
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala
deleted file mode 100644
index f25b5a3..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/ReduceOperator.scala
+++ /dev/null
@@ -1,363 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-import scala.language.reflectiveCalls
-import scala.reflect.macros.Context
-
-import java.util.{ Iterator => JIterator }
-
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.OneInputHintable
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.codegen.{MacroContextHolder, Util}
-import org.apache.flink.api.scala.functions.{ReduceFunction, ReduceFunctionBase, CombinableGroupReduceFunction, GroupReduceFunction}
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.analysis.FieldSelector
-
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.util.Collector
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.java.record.operators.MapOperator
-import org.apache.flink.types.Record
-import org.apache.flink.types.IntValue
-import org.apache.flink.api.java.record.operators.ReduceOperator
-import org.apache.flink.api.java.record.functions.{ReduceFunction => JReduceFunction}
-
-
-class KeyedDataSet[In](val keySelection: List[Int], val input: DataSet[In]) {
-  def reduceGroup[Out](fun: Iterator[In] => Out): DataSet[Out] with OneInputHintable[In, Out] = macro ReduceMacros.reduceGroup[In, Out]
-  def combinableReduceGroup(fun: Iterator[In] => In): DataSet[In] with OneInputHintable[In, In] = macro ReduceMacros.combinableReduceGroup[In]
-  
-  def reduce(fun: (In, In) => In): DataSet[In] with OneInputHintable[In, In] = macro ReduceMacros.reduce[In]
-  
-  def count() : DataSet[(In, Int)] with OneInputHintable[In, (In, Int)] = macro ReduceMacros.count[In]
-}
-
-object ReduceMacros {
-  
-  def groupBy[In: c.WeakTypeTag, Key: c.WeakTypeTag](c: Context { type PrefixType = DataSet[In] })
-                                                    (keyFun: c.Expr[In => Key]): c.Expr[KeyedDataSet[In]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val keySelection = slave.getSelector(keyFun)
-
-    val helper = reify {
-    	new KeyedDataSet[In](keySelection.splice, c.prefix.splice)
-    }
-
-    return helper
-  }
-
-  def reduce[In: c.WeakTypeTag](c: Context { type PrefixType = KeyedDataSet[In] })
-                               (fun: c.Expr[(In, In) => In]): c.Expr[DataSet[In] with OneInputHintable[In, In]] = {
-    import c.universe._
-
-    reduceImpl(c)(c.prefix, fun)
-  }
-
-  def globalReduce[In: c.WeakTypeTag](c: Context { type PrefixType = DataSet[In] })(fun: c.Expr[(In, In) => In]): c.Expr[DataSet[In] with OneInputHintable[In, In]] = {
-    import c.universe._
-
-    reduceImpl(c)(reify { new KeyedDataSet[In](List[Int](), c.prefix.splice) }, fun)
-  }
-
-  def reduceGroup[In: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = KeyedDataSet[In] })
-                                                        (fun: c.Expr[Iterator[In] => Out]): c.Expr[DataSet[Out] with OneInputHintable[In, Out]] = {
-    import c.universe._
-
-    reduceGroupImpl(c)(c.prefix, fun)
-  }
-
-  def globalReduceGroup[In: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context { type PrefixType = DataSet[In] })(fun: c.Expr[Iterator[In] => Out]): c.Expr[DataSet[Out] with OneInputHintable[In, Out]] = {
-    import c.universe._
-
-    reduceGroupImpl(c)(reify { new KeyedDataSet[In](List[Int](), c.prefix.splice) }, fun)
-  }
-
-  def combinableReduceGroup[In: c.WeakTypeTag](c: Context { type PrefixType = KeyedDataSet[In] })
-                                              (fun: c.Expr[Iterator[In] => In]): c.Expr[DataSet[In] with OneInputHintable[In, In]] = {
-    import c.universe._
-
-    combinableReduceGroupImpl(c)(c.prefix, fun)
-  }
-
-  def combinableGlobalReduceGroup[In: c.WeakTypeTag](c: Context { type PrefixType = DataSet[In] })
-                                              (fun: c.Expr[Iterator[In] => In]): c.Expr[DataSet[In] with OneInputHintable[In, In]] = {
-    import c.universe._
-
-    combinableReduceGroupImpl(c)(reify { new KeyedDataSet[In](List[Int](), c.prefix.splice) }, fun)
-  }
-
-  def reduceImpl[In: c.WeakTypeTag](c: Context)
-                               (groupedInput: c.Expr[KeyedDataSet[In]], fun: c.Expr[(In, In) => In]): c.Expr[DataSet[In] with OneInputHintable[In, In]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-//    val (paramName, udfBody) = slave.extractOneInputUdf(fun.tree)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-
-    val stub: c.Expr[ReduceFunctionBase[In, In]] = if (fun.actualType <:< weakTypeOf[ReduceFunction[In]])
-      reify { fun.splice.asInstanceOf[ReduceFunctionBase[In, In]] }
-    else reify {
-      implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice
-
-      new ReduceFunctionBase[In, In] {
-        override def combine(records: JIterator[Record], out: Collector[Record]) = {
-          reduce(records, out)
-        }
-
-        override def reduce(records: JIterator[Record], out: Collector[Record]) = {
-          if (records.hasNext) {
-            val firstRecord = reduceIterator.initialize(records)
-            reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-            val output = reduceIterator.reduce(fun.splice)
-
-            reduceSerializer.serialize(output, reduceRecord)
-            out.collect(reduceRecord)
-          }
-        }
-      }
-
-    }
-    val contract = reify {
-      val helper = groupedInput.splice
-      val input = helper.input.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val keySelection = helper.keySelection
-      val keySelector = new FieldSelector(generatedStub.inputUDT, keySelection)
-
-      val builder = ReduceOperator.builder(generatedStub).input(input)
-
-      val keyPositions = keySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.inputUDT.getKeySet(keyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), keyPositions(i)) }
-      
-      val ret = new ReduceOperator(builder) with OneInputKeyedScalaOperator[In, In] {
-        override val key: FieldSelector = keySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Annotations.getCombinable() +: Seq(
-          Annotations.getConstantFields(
-            Util.filterNonForwards(getUDF.getForwardIndexArrayFrom, getUDF.getForwardIndexArrayTo)))
-      }
-      new DataSet[In](ret) with OneInputHintable[In, In] {}
-    }
-
-    val result = c.Expr[DataSet[In] with OneInputHintable[In, In]](Block(List(udtIn), contract.tree))
-    
-    return result
-  }
-
-  def reduceGroupImpl[In: c.WeakTypeTag, Out: c.WeakTypeTag](c: Context)
-                                                            (groupedInput: c.Expr[KeyedDataSet[In]], fun: c.Expr[Iterator[In] => Out]): c.Expr[DataSet[Out] with OneInputHintable[In, Out]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-//    val (paramName, udfBody) = slave.extractOneInputUdf(fun.tree)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[Out]
-
-    val stub: c.Expr[ReduceFunctionBase[In, Out]] = if (fun.actualType <:< weakTypeOf[GroupReduceFunction[In, Out]])
-      reify { fun.splice.asInstanceOf[ReduceFunctionBase[In, Out]] }
-    else reify {
-      implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice
-      implicit val outputUDT: UDT[Out] = c.Expr[UDT[Out]](createUdtOut).splice
-
-      new ReduceFunctionBase[In, Out] {
-        override def reduce(recordsIterable: JIterator[Record], out: Collector[Record]) = {
-          val records: JIterator[Record] = recordsIterable
-          
-          if (records.hasNext) {
-            val firstRecord = reduceIterator.initialize(records)
-            reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-            val output = fun.splice.apply(reduceIterator)
-
-            reduceSerializer.serialize(output, reduceRecord)
-            out.collect(reduceRecord)
-          }
-        }
-      }
-    }
-    val contract = reify {
-      val helper = groupedInput.splice
-      val input = helper.input.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val keySelection = helper.keySelection
-      val keySelector = new FieldSelector(generatedStub.inputUDT, keySelection)
-      val builder = ReduceOperator.builder(generatedStub).input(input)
-
-      val keyPositions = keySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.inputUDT.getKeySet(keyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), keyPositions(i)) }
-      
-      val ret = new ReduceOperator(builder) with OneInputKeyedScalaOperator[In, Out] {
-        override val key: FieldSelector = keySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Seq(
-          Annotations.getConstantFields(
-            Util.filterNonForwards(getUDF.getForwardIndexArrayFrom, getUDF.getForwardIndexArrayTo)))
-      }
-      new DataSet[Out](ret) with OneInputHintable[In, Out] {}
-    }
-
-    val result = c.Expr[DataSet[Out] with OneInputHintable[In, Out]](Block(List(udtIn, udtOut), contract.tree))
-    
-    return result
-  }
-  
-  def combinableReduceGroupImpl[In: c.WeakTypeTag](c: Context)
-                                              (groupedInput: c.Expr[KeyedDataSet[In]], fun: c.Expr[Iterator[In] => In]): c.Expr[DataSet[In] with OneInputHintable[In, In]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-//    val (paramName, udfBody) = slave.extractOneInputUdf(fun.tree)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-
-    val stub: c.Expr[ReduceFunctionBase[In, In]] = if (fun.actualType <:< weakTypeOf[CombinableGroupReduceFunction[In, In]])
-      reify { fun.splice.asInstanceOf[ReduceFunctionBase[In, In]] }
-    else reify {
-      implicit val inputUDT: UDT[In] = c.Expr[UDT[In]](createUdtIn).splice
-
-      new ReduceFunctionBase[In, In] {
-        override def combine(records: JIterator[Record], out: Collector[Record]) = {
-          reduce(records, out)
-        }
-
-        override def reduce(records: JIterator[Record], out: Collector[Record]) = {
-          val firstRecord = reduceIterator.initialize(records)
-          reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-          val output = fun.splice.apply(reduceIterator)
-
-          reduceSerializer.serialize(output, reduceRecord)
-          out.collect(reduceRecord)
-        }
-      }
-    }
-    val contract = reify {
-      val helper = groupedInput.splice
-      val input = helper.input.contract
-      val generatedStub = ClosureCleaner.clean(stub.splice)
-      val keySelection = helper.keySelection
-      val keySelector = new FieldSelector(generatedStub.inputUDT, keySelection)
-      val builder = ReduceOperator.builder(generatedStub).input(input)
-
-      val keyPositions = keySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.inputUDT.getKeySet(keyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), keyPositions(i)) }
-      
-      val ret = new ReduceOperator(builder) with OneInputKeyedScalaOperator[In, In] {
-        override val key: FieldSelector = keySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Annotations.getCombinable() +: Seq(
-          Annotations.getConstantFields(
-            Util.filterNonForwards(getUDF.getForwardIndexArrayFrom, getUDF.getForwardIndexArrayTo)))
-      }
-      new DataSet[In](ret) with OneInputHintable[In, In] {}
-    }
-
-    val result = c.Expr[DataSet[In] with OneInputHintable[In, In]](Block(List(udtIn), contract.tree))
-    
-    return result
-  }
-
-  def count[In: c.WeakTypeTag](c: Context { type PrefixType = KeyedDataSet[In] })() : c.Expr[DataSet[(In, Int)] with OneInputHintable[In, (In, Int)]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-
-    val (udtIn, createUdtIn) = slave.mkUdtClass[In]
-    val (udtOut, createUdtOut) = slave.mkUdtClass[(In, Int)]
-    
-    val contract = reify {
-      val helper: KeyedDataSet[In] = c.prefix.splice
-      val keySelection = helper.keySelection
-
-      val generatedStub = new JReduceFunction with Serializable {
-        val inputUDT = c.Expr[UDT[In]](createUdtIn).splice
-        val outputUDT = c.Expr[UDT[(In, Int)]](createUdtOut).splice
-        val keySelector = new FieldSelector(inputUDT, keySelection)
-        val udf: UDF1[In, (In, Int)] = new UDF1(inputUDT, outputUDT)
-        
-        private val reduceRecord = new Record()
-        private val pactInt = new IntValue()
-
-        private var countPosition: Int = 0;
-
-        override def open(config: Configuration) = {
-          super.open(config)
-          this.countPosition = udf.getOutputLength - 1;
-        }
-        
-        override def reduce(records: JIterator[Record], result: Collector[Record]) : Unit = {
-          
-          var record : Record = null
-          var counter: Int = 0
-          while (records.hasNext()) {
-            record = records.next()
-            val count = if (record.getNumFields() <= countPosition || record.isNull(countPosition)) 1 else record.getField(countPosition, pactInt).getValue()
-            counter = counter + count
-          }
-          
-          pactInt.setValue(counter)
-          record.setField(countPosition, pactInt)
-          result.collect(record)
-        }
-        
-        override def combine(records: JIterator[Record], result: Collector[Record]) : Unit = {
-          reduce(records, result)
-        }
-
-      }
-      
-      val builder = ReduceOperator.builder(generatedStub).input(helper.input.contract)
-
-      val keyPositions = generatedStub.keySelector.selectedFields.toIndexArray
-      val keyTypes = generatedStub.inputUDT.getKeySet(keyPositions)
-      // global indexes haven't been computed yet...
-      0 until keyTypes.size foreach { i => builder.keyField(keyTypes(i), keyPositions(i)) }
-      
-      val ret = new ReduceOperator(builder) with OneInputKeyedScalaOperator[In, (In, Int)] {
-        override val key: FieldSelector = generatedStub.keySelector
-        override def getUDF = generatedStub.udf
-        override def annotations = Annotations.getCombinable() +: Seq(Annotations.getConstantFieldsExcept(Array[Int]()))
-      }
-      new DataSet[(In, Int)](ret) with OneInputHintable[In, (In, Int)] {}
-    }
-
-    val result = c.Expr[DataSet[(In, Int)] with OneInputHintable[In, (In, Int)]](Block(List(udtIn, udtOut), contract.tree))
-    
-    return result
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/UnionOperator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/UnionOperator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/UnionOperator.scala
deleted file mode 100644
index 5926eef..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/UnionOperator.scala
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators
-
-import language.experimental.macros
-
-import org.apache.flink.api.scala.UnionScalaOperator
-import org.apache.flink.api.scala.DataSet
-import org.apache.flink.api.scala.analysis.UDF2
-
-import org.apache.flink.api.common.operators.Union
-import org.apache.flink.types.Record
-
-object UnionOperator {
-
-  def impl[In](firstInput: DataSet[In], secondInput: DataSet[In]): DataSet[In] = {
-    val union = new Union[Record](firstInput.contract, secondInput.contract)
-      with UnionScalaOperator[In] {
-
-      private val inputUDT = firstInput.contract.getUDF.outputUDT
-      private val udf: UDF2[In, In, In] = new UDF2(inputUDT, inputUDT, inputUDT)
-
-      override def getUDF = udf
-    }
-    new DataSet(union)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/package.scala
deleted file mode 100644
index 5a20940..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/operators/package.scala
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import scala.language.implicitConversions
-
-import scala.collection.TraversableOnce
-
-package object operators {
-
-  implicit def traversableToIterator[T](i: TraversableOnce[T]): Iterator[T] = i.toIterator
-  implicit def optionToIterator[T](opt: Option[T]): Iterator[T] = opt.iterator
-  implicit def arrayToIterator[T](arr: Array[T]): Iterator[T] = arr.iterator
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
new file mode 100644
index 0000000..405158f
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api
+
+import _root_.scala.reflect.ClassTag
+import language.experimental.macros
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.api.scala.typeutils.TypeUtils
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+
+package object scala {
+  // We have this here so that we always have generated TypeInformationS when
+  // using the Scala API
+  implicit def createTypeInformation[T]: TypeInformation[T] = macro TypeUtils.createTypeInfo[T]
+
+  // We need to wrap Java DataSet because we need the scala operations
+  private[flink] def wrap[R: ClassTag](set: JavaDataSet[R]) = new DataSet[R](set)
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
new file mode 100644
index 0000000..3d21f05
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleComparator.scala
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.api.java.typeutils.runtime.TupleComparatorBase
+import org.apache.flink.core.memory.MemorySegment
+import org.apache.flink.types.{KeyFieldOutOfBoundsException, NullKeyFieldException}
+;
+
+/**
+ * Comparator for Scala Tuples. Access is different from
+ * our Java Tuples so we have to treat them differently.
+ */
+class ScalaTupleComparator[T <: Product](
+    keys: Array[Int],
+    scalaComparators: Array[TypeComparator[_]],
+    scalaSerializers: Array[TypeSerializer[_]] )
+  extends TupleComparatorBase[T](keys, scalaComparators, scalaSerializers) {
+
+  private val extractedKeys = new Array[AnyRef](keys.length)
+
+  // We cannot use the Clone Constructor from Scala so we have to do it manually
+  def duplicate: TypeComparator[T] = {
+    // ensure that the serializers are available
+    instantiateDeserializationUtils()
+    val result = new ScalaTupleComparator[T](keyPositions, comparators, serializers)
+    result.privateDuplicate(this)
+    result
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Comparator Methods
+  // --------------------------------------------------------------------------------------------
+
+  def hash(value: T): Int = {
+    val comparator = comparators(0).asInstanceOf[TypeComparator[Any]]
+    var code: Int = comparator.hash(value.productElement(keyPositions(0)))
+    for (i <- 1 until keyPositions.length) {
+      try {
+        code *= TupleComparatorBase.HASH_SALT(i & 0x1F)
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        code += comparator.hash(value.productElement(keyPositions(i)))
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+    code
+  }
+
+  def setReference(toCompare: T) {
+    for (i <- 0 until keyPositions.length) {
+      try {
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        comparator.setReference(toCompare.productElement(keyPositions(i)))
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+  }
+
+  def equalToReference(candidate: T): Boolean = {
+    for (i <- 0 until keyPositions.length) {
+      try {
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        if (!comparator.equalToReference(candidate.productElement(keyPositions(i)))) {
+          return false
+        }
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+    true
+  }
+
+  def compare(first: T, second: T): Int = {
+    for (i <- 0 until keyPositions.length) {
+      try {
+        val keyPos: Int = keyPositions(i)
+        val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+        val cmp: Int = comparator.compare(
+          first.productElement(keyPos),
+          second.productElement(keyPos))
+        if (cmp != 0) {
+          return cmp
+        }
+      } catch {
+        case npex: NullPointerException =>
+          throw new NullKeyFieldException(keyPositions(i))
+        case iobex: IndexOutOfBoundsException =>
+          throw new KeyFieldOutOfBoundsException(keyPositions(i))
+      }
+    }
+    0
+  }
+
+  def putNormalizedKey(value: T, target: MemorySegment, offsetParam: Int, numBytesParam: Int) {
+    var numBytes = numBytesParam
+    var offset = offsetParam
+    var i: Int = 0
+    try {
+      while (i < numLeadingNormalizableKeys && numBytes > 0) {
+        {
+          var len: Int = normalizedKeyLengths(i)
+          len = if (numBytes >= len) len else numBytes
+          val comparator = comparators(i).asInstanceOf[TypeComparator[Any]]
+          comparator.putNormalizedKey(value.productElement(keyPositions(i)), target, offset, len)
+          numBytes -= len
+          offset += len
+        }
+        i += 1
+      }
+    } catch {
+      case npex: NullPointerException => throw new NullKeyFieldException(keyPositions(i))
+    }
+  }
+
+  def extractKeys(value: T) = {
+    for (i <- 0 until keyPositions.length ) {
+      extractedKeys(i) = value.productElement(keyPositions(i)).asInstanceOf[AnyRef]
+    }
+    extractedKeys
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
new file mode 100644
index 0000000..a63bb35
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleSerializer.scala
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase
+import org.apache.flink.core.memory.{DataOutputView, DataInputView}
+;
+
+/**
+ * Serializer for Scala Tuples. Creation and access is different from
+ * our Java Tuples so we have to treat them differently.
+ */
+abstract class ScalaTupleSerializer[T <: Product](
+    tupleClass: Class[T],
+    scalaFieldSerializers: Array[TypeSerializer[_]])
+  extends TupleSerializerBase[T](tupleClass, scalaFieldSerializers) {
+
+  def createInstance: T = {
+    val fields: Array[AnyRef] = new Array(arity)
+    for (i <- 0 until arity) {
+      fields(i) = fieldSerializers(i).createInstance()
+    }
+    createInstance(fields)
+  }
+
+  def copy(from: T, reuse: T): T = {
+    val fields: Array[AnyRef] = new Array(arity)
+    for (i <- 0 until arity) {
+      fields(i) = from.productElement(i).asInstanceOf[AnyRef]
+    }
+    createInstance(fields)
+  }
+
+  def serialize(value: T, target: DataOutputView) {
+    for (i <- 0 until arity) {
+      val serializer = fieldSerializers(i).asInstanceOf[TypeSerializer[Any]]
+      serializer.serialize(value.productElement(i), target)
+    }
+  }
+
+  def deserialize(reuse: T, source: DataInputView): T = {
+    val fields: Array[AnyRef] = new Array(arity)
+    for (i <- 0 until arity) {
+      val field = reuse.productElement(i).asInstanceOf[AnyRef]
+      fields(i) = fieldSerializers(i).deserialize(field, source)
+    }
+    createInstance(fields)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
new file mode 100644
index 0000000..069e03b
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaTupleTypeInfo.scala
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.api.java.typeutils.{TupleTypeInfo, AtomicType, TupleTypeInfoBase}
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+
+/**
+ * TypeInformation for Scala Tuples. Creation and access is different from
+ * our Java Tuples so we have to treat them differently.
+ */
+abstract class ScalaTupleTypeInfo[T <: Product](
+    tupleClass: Class[T],
+    fieldTypes: Seq[TypeInformation[_]])
+  extends TupleTypeInfoBase[T](tupleClass, fieldTypes: _*) {
+
+  def createComparator(logicalKeyFields: Array[Int], orders: Array[Boolean]): TypeComparator[T] = {
+    // sanity checks
+    if (logicalKeyFields == null || orders == null
+      || logicalKeyFields.length != orders.length || logicalKeyFields.length > types.length) {
+      throw new IllegalArgumentException
+    }
+
+    // No special handling of leading Key field as in JavaTupleComparator for now
+
+    // --- general case ---
+    var maxKey: Int = -1
+
+    for (key <- logicalKeyFields) {
+      maxKey = Math.max(key, maxKey)
+    }
+
+    if (maxKey >= types.length) {
+      throw new IllegalArgumentException("The key position " + maxKey + " is out of range for " +
+        "Tuple" + types.length)
+    }
+
+    // create the comparators for the individual fields
+    val fieldComparators: Array[TypeComparator[_]] = new Array(logicalKeyFields.length)
+
+    for (i <- 0 until logicalKeyFields.length) {
+      val keyPos = logicalKeyFields(i)
+      if (types(keyPos).isKeyType && types(keyPos).isInstanceOf[AtomicType[_]]) {
+        fieldComparators(i) = types(keyPos).asInstanceOf[AtomicType[_]].createComparator(orders(i))
+      } else {
+        throw new IllegalArgumentException(
+          "The field at position " + i + " (" + types(keyPos) + ") is no atomic key type.")
+      }
+    }
+
+    // create the serializers for the prefix up to highest key position
+    val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](maxKey + 1)
+
+    for (i <- 0 to maxKey) {
+      fieldSerializers(i) = types(i).createSerializer
+    }
+
+    new ScalaTupleComparator[T](logicalKeyFields, fieldComparators, fieldSerializers)
+  }
+
+  override def toString = "Scala " + super.toString
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
new file mode 100644
index 0000000..5901e48
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TypeUtils.scala
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.typeutils
+
+import scala.reflect.macros.Context
+import org.apache.flink.api.scala.codegen.MacroContextHolder
+import org.apache.flink.types.TypeInformation
+
+private[flink] object TypeUtils {
+
+  def createTypeInfo[T: c.WeakTypeTag](c: Context): c.Expr[TypeInformation[T]] = {
+    val slave = MacroContextHolder.newMacroHelper(c)
+    slave.mkTypeInfo[T]
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
new file mode 100644
index 0000000..f8cbb62
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala
+
+import org.apache.flink.api.common.InvalidProgramException
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.operators.Keys
+import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
+import org.apache.flink.types.TypeInformation
+
+/**
+ * This is for dealing with operations that require keys and use a fluent interface (join, and
+ * coGroup for now). For each operation we need a subclass that implements `finish` to
+ * create the actual operation using the provided keys.
+ *
+ * This way, we have a central point where all the key-providing happens and don't need to change
+ * the specific operations if the supported key types change.
+ *
+ * We use the type parameter `R` to specify the type of the resulting operation. For join
+ * this would be `JoinDataSet[T, O]` and for coGroup it would be `CoGroupDataSet[T, O]`. This
+ * way the user gets the correct type for the finished operation.
+ *
+ * @tparam T Type of the left input [[DataSet]].
+ * @tparam O Type of the right input [[DataSet]].
+ * @tparam R The type of the resulting Operation.
+ */
+private[flink] abstract class UnfinishedKeyPairOperation[T, O, R](
+    private[flink] val leftSet: JavaDataSet[T],
+    private[flink] val rightSet: JavaDataSet[O]) {
+
+  private[flink] def finish(leftKey: Keys[T], rightKey: Keys[O]): R
+
+  /**
+   * Specify the key fields for the left side of the key based operation. This returns
+   * a [[HalfUnfinishedKeyPairOperation]] on which `isEqualTo` must be called to specify the
+   * key for the right side. The result after specifying the right side key is the finished
+   * operation.
+   *
+   * This only works on a Tuple [[DataSet]].
+   */
+  def where(leftKeys: Int*) = {
+    val leftKey = new FieldPositionKeys[T](leftKeys.toArray, leftSet.getType)
+    new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)
+  }
+
+  /**
+   * Specify the key selector function for the left side of the key based operation. This returns
+   * a [[HalfUnfinishedKeyPairOperation]] on which `isEqualTo` must be called to specify the
+   * key for the right side. The result after specifying the right side key is the finished
+   * operation.
+   */
+  def where[K: TypeInformation](fun: (T) => K) = {
+    val keyType = implicitly[TypeInformation[K]]
+    val keyExtractor = new KeySelector[T, K] {
+      def getKey(in: T) = fun(in)
+    }
+    val leftKey = new Keys.SelectorFunctionKeys[T, K](keyExtractor, leftSet.getType, keyType)
+    new HalfUnfinishedKeyPairOperation[T, O, R](this, leftKey)
+  }
+}
+
+private[flink] class HalfUnfinishedKeyPairOperation[T, O, R](
+    unfinished: UnfinishedKeyPairOperation[T, O, R], leftKey: Keys[T]) {
+
+  /**
+   * Specify the key fields for the right side of the key based operation. This returns
+   * the finished operation.
+   *
+   * This only works on a Tuple [[DataSet]].
+   */
+  def equalTo(rightKeys: Int*): R = {
+    val rightKey = new FieldPositionKeys[O](rightKeys.toArray, unfinished.rightSet.getType)
+    if (!leftKey.areCompatibale(rightKey)) {
+      throw new InvalidProgramException("The types of the key fields do not match. Left: " +
+        leftKey + " Right: " + rightKey)
+    }
+    unfinished.finish(leftKey, rightKey)
+  }
+
+  /**
+   * Specify the key selector function for the right side of the key based operation. This returns
+   * the finished operation.
+   */
+  def equalTo[K: TypeInformation](fun: (O) => K) = {
+    val keyType = implicitly[TypeInformation[K]]
+    val keyExtractor = new KeySelector[O, K] {
+      def getKey(in: O) = fun(in)
+    }
+    val rightKey =
+      new Keys.SelectorFunctionKeys[O, K](keyExtractor, unfinished.rightSet.getType, keyType)
+    if (!leftKey.areCompatibale(rightKey)) {
+      throw new InvalidProgramException("The types of the key fields do not match. Left: " +
+        leftKey + " Right: " + rightKey)
+    }
+    unfinished.finish(leftKey, rightKey)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/CollectionDataSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/CollectionDataSourceTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/CollectionDataSourceTest.scala
deleted file mode 100644
index 19feb3a..0000000
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/CollectionDataSourceTest.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import org.apache.flink.api.java.record.operators.{CollectionDataSource => JCollectionDataSource}
-import org.apache.flink.types.{DoubleValue, Record}
-import org.scalatest.junit.AssertionsForJUnit
-import org.junit.Assert._
-import org.junit.Test
-
-
-class CollectionDataSourceTest extends AssertionsForJUnit {
-  @Test def testScalaCollectionInput() {
-    val expected = List(1.0, 2.0, 3.0)
-    val datasource = org.apache.flink.api.scala.CollectionDataSource(expected)
-
-    val javaCDS = datasource.contract.asInstanceOf[JCollectionDataSource]
-
-    val inputFormat = javaCDS.getFormatWrapper.getUserCodeObject()
-    val splits = inputFormat.createInputSplits(1)
-    inputFormat.open(splits(0))
-
-    val record = new Record()
-    var result = List[Double]()
-
-    while(!inputFormat.reachedEnd()){
-      inputFormat.nextRecord(record)
-      assertTrue(record.getNumFields == 1)
-      val value = record.getField[DoubleValue](0, classOf[DoubleValue])
-      result = value.getValue :: result
-    }
-
-    assertEquals(expected, result.reverse)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
index b4927b4..d5b0d24 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
@@ -1,192 +1,192 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.scala
-
-import org.junit.Test
-import org.apache.flink.api.common.InvalidProgramException
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-import org.scalatest.junit.AssertionsForJUnit
-
-// Verify that the sanity checking in delta iterations works. We just
-// have a dummy job that is not meant to be executed. Only verify that
-// the join/coGroup inside the iteration is checked.
-class DeltaIterationSanityCheckTest extends Serializable {
-
-  @Test
-  def testCorrectJoinWithSolution1 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = s join ws where {_._1} isEqualTo {_._1} map { (l, r) => l }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test
-  def testCorrectJoinWithSolution2 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = ws join s where {_._1} isEqualTo {_._1} map { (l, r) => l }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test(expected = classOf[InvalidProgramException])
-  def testIncorrectJoinWithSolution1 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = s join ws where {_._2} isEqualTo {_._2} map { (l, r) => l }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test(expected = classOf[InvalidProgramException])
-  def testIncorrectJoinWithSolution2 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = ws join s where {_._2} isEqualTo {_._2} map { (l, r) => l }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test(expected = classOf[InvalidProgramException])
-  def testIncorrectJoinWithSolution3 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = s join ws where {_._1} isEqualTo {_._1} map { (l, r) => l }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._2}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-   }
-
-  @Test
-  def testCorrectCoGroupWithSolution1 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = s cogroup ws where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test
-  def testCorrectCoGroupWithSolution2 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = ws cogroup s where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test(expected = classOf[InvalidProgramException])
-  def testIncorrectCoGroupWithSolution1 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = s cogroup ws where {_._2} isEqualTo {_._2} map { (l, r) => l.next() }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test(expected = classOf[InvalidProgramException])
-  def testIncorrectCoGroupWithSolution2 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = ws cogroup s where {_._2} isEqualTo {_._2} map { (l, r) => l.next() }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-
-  @Test(expected = classOf[InvalidProgramException])
-  def testIncorrectCoGroupWithSolution3 {
-    val solutionInput = CollectionDataSource(Array((1, "1")))
-    val worksetInput = CollectionDataSource(Array((2, "2")))
-
-    def step(s: DataSet[(Int, String)], ws: DataSet[(Int, String)]) = {
-      val result = s cogroup ws where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
-      (result, ws)
-    }
-    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._2}, step, 10)
-
-    val output = iteration.write("/dummy", CsvOutputFormat())
-
-    val plan = new ScalaPlan(Seq(output))
-  }
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.flink.api.scala
+//
+//import org.junit.Test
+//import org.apache.flink.api.common.InvalidProgramException
+//
+//import org.apache.flink.api.scala._
+//import org.apache.flink.api.scala.operators._
+//import org.scalatest.junit.AssertionsForJUnit
+//
+//// Verify that the sanity checking in delta iterations works. We just
+//// have a dummy job that is not meant to be executed. Only verify that
+//// the join/coGroup inside the iteration is checked.
+//class DeltaIterationSanityCheckTest extends Serializable {
+//
+//  @Test
+//  def testCorrectJoinWithSolution1 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = s join ws where {_._1} isEqualTo {_._1} map { (l, r) => l }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test
+//  def testCorrectJoinWithSolution2 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = ws join s where {_._1} isEqualTo {_._1} map { (l, r) => l }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test(expected = classOf[InvalidProgramException])
+//  def testIncorrectJoinWithSolution1 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = s join ws where {_._2} isEqualTo {_._2} map { (l, r) => l }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test(expected = classOf[InvalidProgramException])
+//  def testIncorrectJoinWithSolution2 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = ws join s where {_._2} isEqualTo {_._2} map { (l, r) => l }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test(expected = classOf[InvalidProgramException])
+//  def testIncorrectJoinWithSolution3 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = s join ws where {_._1} isEqualTo {_._1} map { (l, r) => l }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._2}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//   }
+//
+//  @Test
+//  def testCorrectCoGroupWithSolution1 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = s cogroup ws where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test
+//  def testCorrectCoGroupWithSolution2 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = ws cogroup s where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test(expected = classOf[InvalidProgramException])
+//  def testIncorrectCoGroupWithSolution1 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = s cogroup ws where {_._2} isEqualTo {_._2} map { (l, r) => l.next() }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test(expected = classOf[InvalidProgramException])
+//  def testIncorrectCoGroupWithSolution2 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = ws cogroup s where {_._2} isEqualTo {_._2} map { (l, r) => l.next() }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//
+//  @Test(expected = classOf[InvalidProgramException])
+//  def testIncorrectCoGroupWithSolution3 {
+//    val solutionInput = CollectionDataSource(Array((1, "1")))
+//    val worksetInput = CollectionDataSource(Array((2, "2")))
+//
+//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
+//      val result = s cogroup ws where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
+//      (result, ws)
+//    }
+//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._2}, step, 10)
+//
+//    val output = iteration.write("/dummy", CsvOutputFormat())
+//
+//    val plan = new ScalaPlan(Seq(output))
+//  }
+//}


[50/60] git commit: Bump doc version to 0.7-incubating

Posted by al...@apache.org.
Bump doc version to 0.7-incubating


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

Branch: refs/heads/master
Commit: 15060ef49e142461df5078133d3109fafc819f92
Parents: 7084fa2
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Sep 19 11:56:37 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 docs/_config.yml               | 12 ++++++------
 docs/run_example_quickstart.md |  2 +-
 2 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/15060ef4/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 95841ca..6652886 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -5,15 +5,15 @@
 #     {{ site.CONFIG_KEY }}
 #------------------------------------------------------------------------------
 
-FLINK_VERSION_STABLE: 0.6-incubating # this variable can point to a SNAPSHOT version in the git source.
-FLINK_VERSION_HADOOP_2_STABLE: 0.6-hadoop2-incubating
-FLINK_VERSION_SHORT: 0.6
+FLINK_VERSION_STABLE: 0.7-incubating # this variable can point to a SNAPSHOT version in the git source.
+FLINK_VERSION_HADOOP_2_STABLE: 0.7-hadoop2-incubating
+FLINK_VERSION_SHORT: 0.7
 FLINK_ISSUES_URL: https://issues.apache.org/jira/browse/FLINK
 FLINK_GITHUB_URL:  https://github.com/apache/incubator-flink
 
-FLINK_DOWNLOAD_URL_HADOOP_1_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.6-incubating-bin-hadoop1.tgz
-FLINK_DOWNLOAD_URL_HADOOP_2_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.6-incubating-bin-hadoop2.tgz
-FLINK_DOWNLOAD_URL_YARN_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.6-incubating-bin-hadoop2-yarn.tgz
+FLINK_DOWNLOAD_URL_HADOOP_1_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.7-incubating-bin-hadoop1.tgz
+FLINK_DOWNLOAD_URL_HADOOP_2_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.7-incubating-bin-hadoop2.tgz
+FLINK_DOWNLOAD_URL_YARN_STABLE: http://www.apache.org/dyn/closer.cgi/incubator/flink/flink-0.7-incubating-bin-hadoop2-yarn.tgz
 
 #------------------------------------------------------------------------------
 # BUILD CONFIG

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/15060ef4/docs/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/run_example_quickstart.md b/docs/run_example_quickstart.md
index 88d6137..7fabaab 100644
--- a/docs/run_example_quickstart.md
+++ b/docs/run_example_quickstart.md
@@ -76,7 +76,7 @@ The Flink webclient allows to submit Flink programs using a graphical user inter
 		1. <a href="http://localhost:8080/launch.html">Open webclient on localhost:8080</a> <br>
 		2. Upload the file. 
 			{% highlight bash %}
-			examples/flink-java-examples-0.6-incubating-KMeans.jar
+			examples/flink-java-examples-{{ site.FLINK_VERSION_STABLE }}-KMeans.jar
 			{% endhighlight %} </br>
 		3. Select it in the left box to see how the operators in the plan are connected to each other. <br>
 		4. Enter the arguments in the lower left box:


[09/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
index 58a9a2a..b6193aa 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
@@ -53,6 +53,22 @@ public final class TupleSerializer<T extends Tuple> extends TupleSerializerBase<
 	}
 
 	@Override
+	public T createInstance(Object[] fields) {
+		try {
+			T t = tupleClass.newInstance();
+
+			for (int i = 0; i < arity; i++) {
+				t.setField(fields[i], i);
+			}
+
+			return t;
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Cannot instantiate tuple.", e);
+		}
+	}
+
+	@Override
 	public T copy(T from, T reuse) {
 		for (int i = 0; i < arity; i++) {
 			Object copy = fieldSerializers[i].copy(from.getField(i), reuse.getField(i));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
index 69133b6..3e630fd 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
@@ -69,6 +69,14 @@ public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
 		return -1;
 	}
 
+	public int getArity() {
+		return arity;
+	}
+
+	// We use this in the Aggregate and Distinct Operators to create instances
+	// of immutable Typles (i.e. Scala Tuples)
+	public abstract T createInstance(Object[] fields);
+
 	@Override
 	public void copy(DataInputView source, DataOutputView target) throws IOException {
 		for (int i = 0; i < arity; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index 3a88565..fdda028 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -46,9 +46,7 @@ under the License.
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
   </properties>
 
-  <!--  These two requirements are the minimum to use and develop Flink.
-        You can add others like <artifactId>flink-scala</artifactId> for Scala!
-  -->
+  <!--  These two requirements are the minimum to use and develop Flink. -->
   <dependencies>
     <dependency>
       <groupId>org.apache.flink</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala
index 0c29856..f9dad03 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala
@@ -1,108 +1,53 @@
-package ${package};
+package ${package}
 
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.flink.api.common.Program
-import org.apache.flink.api.common.ProgramDescription
-import org.apache.flink.client.LocalExecutor
-import org.apache.flink.api.scala.TextFile
-import org.apache.flink.api.scala.ScalaPlan
 import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.operators._
-import org.apache.flink.client.RemoteExecutor
-
-// You can run this locally using:
-// mvn exec:exec -Dexec.executable="java" -Dexec.args="-cp %classpath ${package}.RunJobLocal 2 file:///some/path file:///some/other/path"
-object RunJobLocal {
-  def main(args: Array[String]) {
-    val job = new Job
-    if (args.size < 3) {
-      println(job.getDescription)
-      return
-    }
-    val plan = job.getScalaPlan(args(0).toInt, args(1), args(2))
-    LocalExecutor.execute(plan)
-    System.exit(0)
-  }
-}
-
-// You can run this on a cluster using:
-// mvn exec:exec -Dexec.executable="java" -Dexec.args="-cp %classpath ${package}.RunJobRemote 2 file:///some/path file:///some/other/path"
-object RunJobRemote {
-  def main(args: Array[String]) {
-    val job = new Job
-    if (args.size < 3) {
-      println(job.getDescription)
-      return
-    }
-    val plan = job.getScalaPlan(args(0).toInt, args(1), args(2))
-    // This will create an executor to run the plan on a cluster. We assume
-    // that the JobManager is running on the local machine on the default
-    // port. Change this according to your configuration.
-    // You will also need to change the name of the jar if you change the
-    // project name and/or version. Before running this you also need
-    // to run "mvn package" to create the jar.
-    val ex = new RemoteExecutor("localhost", 6123, "target/flink-project-0.1-SNAPSHOT.jar")
-    ex.executePlan(plan)
-  }
-}
-
 
 /**
- * This is a outline for a Flink scala job. It is actually the WordCount
- * example from the here distribution.
+ * Skeleton for a Flink Job.
  *
- * You can run it out of your IDE using the main() method of RunJob.
- * This will use the LocalExecutor to start a little Flink instance
- * out of your IDE.
+ * For a full example of a Flink Job, see the WordCountJob.scala file in the
+ * same package/directory or have a look at the website.
  *
  * You can also generate a .jar file that you can submit on your Flink
- * cluster.
- * Just type
- *      mvn clean package
- * in the projects root directory.
- * You will find the jar in
- *      target/flink-quickstart-0.1-SNAPSHOT-Sample.jar
+ * cluster. Just type
+ * {{{
+ *   mvn clean package
+ * }}}
+ * in the projects root directory. You will find the jar in
+ * target/flink-quickstart-0.1-SNAPSHOT-Sample.jar
  *
  */
-class Job extends Program with ProgramDescription with Serializable {
-  override def getDescription() = {
-    "Parameters: [numSubStasks] [input] [output]"
-  }
-  override def getPlan(args: String*) = {
-    getScalaPlan(args(0).toInt, args(1), args(2))
-  }
-
-  def formatOutput = (word: String, count: Int) => "%s %d".format(word, count)
-
-  def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
-    val input = TextFile(textInput)
-
-    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } map { (_, 1) } }
-    val counts = words groupBy { case (word, _) => word } reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
-
-    counts neglects { case (word, _) => word }
-    counts preserves({ case (word, _) => word }, { case (word, _) => word })
-    val output = counts.write(wordsOutput, DelimitedOutputFormat(formatOutput.tupled))
-
-    val plan = new ScalaPlan(Seq(output), "Word Count (immutable)")
-    plan.setDefaultParallelism(numSubTasks)
-    plan
+object Job {
+  def main(args: Array[String]) {
+    // set up the execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    /**
+     * Here, you can start creating your execution plan for Flink.
+     *
+     * Start with getting some data from the environment, like
+     * env.readTextFile(textPath);
+     *
+     * then, transform the resulting DataSet[String] using operations
+     * like:
+     *   .filter()
+     *   .flatMap()
+     *   .join()
+     *   .group()
+     *
+     * and many more.
+     * Have a look at the programming guide for the Scala API:
+     *
+     * http://flink.incubator.apache.org/docs/0.6-SNAPSHOT/java_api_guide.html
+     *
+     * and the examples
+     *
+     * http://flink.incubator.apache.org/docs/0.6-SNAPSHOT/java_api_examples.html
+     *
+     */
+
+
+    // execute program
+    env.execute("Flink Scala API Skeleton")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCountJob.scala
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCountJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCountJob.scala
new file mode 100644
index 0000000..54408cb
--- /dev/null
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/WordCountJob.scala
@@ -0,0 +1,37 @@
+package ${package}
+
+import org.apache.flink.api.scala._
+
+/**
+ * Implements the "WordCount" program that computes a simple word occurrence histogram
+ * over some sample data
+ *
+ * This example shows how to:
+ *
+ *   - write a simple Flink program.
+ *   - use Tuple data types.
+ *   - write and use user-defined functions.
+ */
+object WordCountJob {
+  def main(args: Array[String]) {
+
+    // set up the execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    // get input data
+    val text = env.fromElements("To be, or not to be,--that is the question:--",
+      "Whether 'tis nobler in the mind to suffer", "The slings and arrows of outrageous fortune",
+      "Or to take arms against a sea of troubles,")
+
+    val counts = text.flatMap { _.toLowerCase.split("\\W+") }
+      .map { (_, 1) }
+      .groupBy(0)
+      .sum(1)
+
+    // emit result
+    counts.print()
+
+    // execute program
+    env.execute("WordCount Example")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index 6ce737c..45db390 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -39,7 +39,23 @@ under the License.
 			<version>${project.version}</version>
 		</dependency>
 
-		<dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-core</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-java</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-java</artifactId>
 			<version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/java/org/apache/flink/api/scala/operators/Annotations.java
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/Annotations.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/Annotations.java
deleted file mode 100644
index ec7e9b7..0000000
--- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/Annotations.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.operators;
-
-import java.lang.annotation.Annotation;
-import java.util.Arrays;
-
-import org.apache.flink.api.java.record.functions.FunctionAnnotation;
-import org.apache.flink.api.java.record.operators.ReduceOperator;
-
-public class Annotations {
-
-	public static Annotation getConstantFields(int[] fields) {
-		return new ConstantFields(fields);
-	}
-
-	public static Annotation getConstantFieldsFirst(int[] fields) {
-		return new ConstantFieldsFirst(fields);
-	}
-
-	public static Annotation getConstantFieldsSecond(int[] fields) {
-		return new ConstantFieldsSecond(fields);
-	}
-
-	public static Annotation getConstantFieldsExcept(int[] fields) {
-		return new ConstantFieldsExcept(fields);
-	}
-
-	public static Annotation getConstantFieldsFirstExcept(int[] fields) {
-		return new ConstantFieldsFirstExcept(fields);
-	}
-
-	public static Annotation getConstantFieldsSecondExcept(int[] fields) {
-		return new ConstantFieldsSecondExcept(fields);
-	}
-
-	public static Annotation getCombinable() {
-		return new Combinable();
-	}
-
-	private static abstract class Fields<T extends Annotation> implements Annotation {
-
-		private final Class<T> clazz;
-
-		private final int[] fields;
-
-		public Fields(Class<T> clazz, int[] fields) {
-			this.clazz = clazz;
-			this.fields = fields;
-		}
-
-		public int[] value() {
-			return fields;
-		}
-
-		@Override
-		public Class<? extends Annotation> annotationType() {
-			return clazz;
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public boolean equals(Object obj) {
-			if (obj == null || !annotationType().isAssignableFrom(obj.getClass())) {
-				return false;
-			}
-
-			if (!annotationType().equals(((Annotation) obj).annotationType())) {
-				return false;
-			}
-
-			int[] otherFields = getOtherFields((T) obj);
-			return Arrays.equals(fields, otherFields);
-		}
-
-		protected abstract int[] getOtherFields(T other);
-
-		@Override
-		public int hashCode() {
-			return 0xf16cd51b ^ Arrays.hashCode(fields);
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class ConstantFields extends Fields<FunctionAnnotation.ConstantFields> implements
-			FunctionAnnotation.ConstantFields {
-
-		public ConstantFields(int[] fields) {
-			super(FunctionAnnotation.ConstantFields.class, fields);
-		}
-
-		@Override
-		protected int[] getOtherFields(FunctionAnnotation.ConstantFields other) {
-			return other.value();
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class ConstantFieldsFirst extends Fields<FunctionAnnotation.ConstantFieldsFirst> implements
-			FunctionAnnotation.ConstantFieldsFirst {
-
-		public ConstantFieldsFirst(int[] fields) {
-			super(FunctionAnnotation.ConstantFieldsFirst.class, fields);
-		}
-
-		@Override
-		protected int[] getOtherFields(FunctionAnnotation.ConstantFieldsFirst other) {
-			return other.value();
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class ConstantFieldsSecond extends Fields<FunctionAnnotation.ConstantFieldsSecond> implements
-			FunctionAnnotation.ConstantFieldsSecond {
-
-		public ConstantFieldsSecond(int[] fields) {
-			super(FunctionAnnotation.ConstantFieldsSecond.class, fields);
-		}
-
-		@Override
-		protected int[] getOtherFields(FunctionAnnotation.ConstantFieldsSecond other) {
-			return other.value();
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class ConstantFieldsExcept extends Fields<FunctionAnnotation.ConstantFieldsExcept> implements
-			FunctionAnnotation.ConstantFieldsExcept {
-
-		public ConstantFieldsExcept(int[] fields) {
-			super(FunctionAnnotation.ConstantFieldsExcept.class, fields);
-		}
-
-		@Override
-		protected int[] getOtherFields(FunctionAnnotation.ConstantFieldsExcept other) {
-			return other.value();
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class ConstantFieldsFirstExcept extends Fields<FunctionAnnotation.ConstantFieldsFirstExcept> implements
-			FunctionAnnotation.ConstantFieldsFirstExcept {
-
-		public ConstantFieldsFirstExcept(int[] fields) {
-			super(FunctionAnnotation.ConstantFieldsFirstExcept.class, fields);
-		}
-
-		@Override
-		protected int[] getOtherFields(FunctionAnnotation.ConstantFieldsFirstExcept other) {
-			return other.value();
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class ConstantFieldsSecondExcept extends Fields<FunctionAnnotation.ConstantFieldsSecondExcept> implements
-			FunctionAnnotation.ConstantFieldsSecondExcept {
-
-		public ConstantFieldsSecondExcept(int[] fields) {
-			super(FunctionAnnotation.ConstantFieldsSecondExcept.class, fields);
-		}
-
-		@Override
-		protected int[] getOtherFields(FunctionAnnotation.ConstantFieldsSecondExcept other) {
-			return other.value();
-		}
-	}
-
-	@SuppressWarnings("all")
-	private static class Combinable implements Annotation, ReduceOperator.Combinable {
-
-		public Combinable() {
-		}
-
-		@Override
-		public Class<? extends Annotation> annotationType() {
-			return ReduceOperator.Combinable.class;
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj == null || !annotationType().isAssignableFrom(obj.getClass())) {
-				return false;
-			}
-
-			if (!annotationType().equals(((Annotation) obj).annotationType())) {
-				return false;
-			}
-
-			return true;
-		}
-
-		@Override
-		public int hashCode() {
-			return 0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
new file mode 100644
index 0000000..82a1dd5
--- /dev/null
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
@@ -0,0 +1,316 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.operators;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.lang3.Validate;
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.SingleInputSemanticProperties;
+import org.apache.flink.api.common.operators.UnaryOperatorInformation;
+import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.api.java.aggregation.AggregationFunction;
+import org.apache.flink.api.java.aggregation.AggregationFunctionFactory;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction.Combinable;
+import org.apache.flink.api.java.operators.Grouping;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.operators.SingleInputOperator;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
+import org.apache.flink.api.java.typeutils.runtime.RuntimeStatefulSerializerFactory;
+import org.apache.flink.api.java.typeutils.runtime.RuntimeStatelessSerializerFactory;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Collector;
+
+import scala.Product;
+
+/**
+ * This operator represents the application of a "aggregate" operation on a data set, and the
+ * result data set produced by the function.
+ *
+ * @param <IN> The type of the data set aggregated by the operator.
+ */
+public class ScalaAggregateOperator<IN> extends SingleInputOperator<IN, IN, ScalaAggregateOperator<IN>> {
+
+	private final List<AggregationFunction<?>> aggregationFunctions = new ArrayList<AggregationFunction<?>>(4);
+
+	private final List<Integer> fields = new ArrayList<Integer>(4);
+
+	private final Grouping<IN> grouping;
+
+	/**
+	 * <p>
+	 * Non grouped aggregation
+	 */
+	public ScalaAggregateOperator(org.apache.flink.api.java.DataSet<IN> input, Aggregations function, int field) {
+		super(Validate.notNull(input), input.getType());
+
+		Validate.notNull(function);
+
+		if (!input.getType().isTupleType()) {
+			throw new InvalidProgramException("Aggregating on field positions is only possible on tuple data types.");
+		}
+
+		TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) input.getType();
+
+		if (field < 0 || field >= inType.getArity()) {
+			throw new IllegalArgumentException("Aggregation field position is out of range.");
+		}
+
+		AggregationFunctionFactory factory = function.getFactory();
+		AggregationFunction<?> aggFunct = factory.createAggregationFunction(inType.getTypeAt(field).getTypeClass());
+
+		// this is the first aggregation operator after a regular data set (non grouped aggregation)
+		this.aggregationFunctions.add(aggFunct);
+		this.fields.add(field);
+		this.grouping = null;
+	}
+
+	/**
+	 *
+	 * Grouped aggregation
+	 *
+	 * @param input
+	 * @param function
+	 * @param field
+	 */
+	public ScalaAggregateOperator(Grouping<IN> input, Aggregations function, int field) {
+		super(Validate.notNull(input).getDataSet(), input.getDataSet().getType());
+
+		Validate.notNull(function);
+
+		if (!input.getDataSet().getType().isTupleType()) {
+			throw new InvalidProgramException("Aggregating on field positions is only possible on tuple data types.");
+		}
+
+		TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) input.getDataSet().getType();
+
+		if (field < 0 || field >= inType.getArity()) {
+			throw new IllegalArgumentException("Aggregation field position is out of range.");
+		}
+
+		AggregationFunctionFactory factory = function.getFactory();
+		AggregationFunction<?> aggFunct = factory.createAggregationFunction(inType.getTypeAt(field).getTypeClass());
+
+		// set the aggregation fields
+		this.aggregationFunctions.add(aggFunct);
+		this.fields.add(field);
+		this.grouping = input;
+	}
+
+
+	public ScalaAggregateOperator<IN> and(Aggregations function, int field) {
+		Validate.notNull(function);
+
+		TupleTypeInfoBase<?> inType = (TupleTypeInfoBase<?>) getType();
+
+		if (field < 0 || field >= inType.getArity()) {
+			throw new IllegalArgumentException("Aggregation field position is out of range.");
+		}
+
+
+		AggregationFunctionFactory factory = function.getFactory();
+		AggregationFunction<?> aggFunct = factory.createAggregationFunction(inType.getTypeAt(field).getTypeClass());
+
+		this.aggregationFunctions.add(aggFunct);
+		this.fields.add(field);
+
+		return this;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>> translateToDataFlow(Operator<IN> input) {
+
+		// sanity check
+		if (this.aggregationFunctions.isEmpty() || this.aggregationFunctions.size() != this.fields.size()) {
+			throw new IllegalStateException();
+		}
+
+
+		// construct the aggregation function
+		AggregationFunction<Object>[] aggFunctions = new AggregationFunction[this.aggregationFunctions.size()];
+		int[] fields = new int[this.fields.size()];
+		StringBuilder genName = new StringBuilder();
+
+		for (int i = 0; i < fields.length; i++) {
+			aggFunctions[i] = (AggregationFunction<Object>) this.aggregationFunctions.get(i);
+			fields[i] = this.fields.get(i);
+
+			genName.append(aggFunctions[i].toString()).append('(').append(fields[i]).append(')').append(',');
+		}
+		genName.setLength(genName.length()-1);
+
+		TypeSerializer<IN> serializer = getInputType().createSerializer();
+		TypeSerializerFactory<IN> serializerFactory = null;
+		if (serializer.isStateful()) {
+			serializerFactory = new RuntimeStatefulSerializerFactory<IN>(
+					serializer, getInputType().getTypeClass());
+		} else {
+			serializerFactory = new RuntimeStatelessSerializerFactory<IN>(
+					serializer, getInputType().getTypeClass());
+		}
+
+		@SuppressWarnings("rawtypes")
+		RichGroupReduceFunction<IN, IN> function = new AggregatingUdf(serializerFactory, aggFunctions, fields);
+
+
+		String name = getName() != null ? getName() : genName.toString();
+
+		// distinguish between grouped reduce and non-grouped reduce
+		if (this.grouping == null) {
+			// non grouped aggregation
+			UnaryOperatorInformation<IN, IN> operatorInfo = new UnaryOperatorInformation<IN, IN>(getInputType(), getResultType());
+			GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>> po =
+					new GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>>(function, operatorInfo, new int[0], name);
+
+			po.setCombinable(true);
+
+			// set input
+			po.setInput(input);
+			// set dop
+			po.setDegreeOfParallelism(this.getParallelism());
+
+			return po;
+		}
+
+		if (this.grouping.getKeys() instanceof Keys.FieldPositionKeys) {
+			// grouped aggregation
+			int[] logicalKeyPositions = this.grouping.getKeys().computeLogicalKeyPositions();
+			UnaryOperatorInformation<IN, IN> operatorInfo = new UnaryOperatorInformation<IN, IN>(getInputType(), getResultType());
+			GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>> po =
+					new GroupReduceOperatorBase<IN, IN, GroupReduceFunction<IN, IN>>(function, operatorInfo, logicalKeyPositions, name);
+
+			po.setCombinable(true);
+
+			// set input
+			po.setInput(input);
+			// set dop
+			po.setDegreeOfParallelism(this.getParallelism());
+
+			SingleInputSemanticProperties props = new SingleInputSemanticProperties();
+
+			for (int i = 0; i < logicalKeyPositions.length; i++) {
+				int keyField = logicalKeyPositions[i];
+				boolean keyFieldUsedInAgg = false;
+
+				for (int k = 0; k < fields.length; k++) {
+					int aggField = fields[k];
+					if (keyField == aggField) {
+						keyFieldUsedInAgg = true;
+						break;
+					}
+				}
+
+				if (!keyFieldUsedInAgg) {
+					props.addForwardedField(keyField, keyField);
+				}
+			}
+
+			po.setSemanticProperties(props);
+
+			return po;
+		}
+		else if (this.grouping.getKeys() instanceof Keys.SelectorFunctionKeys) {
+			throw new UnsupportedOperationException("Aggregate does not support grouping with KeySelector functions, yet.");
+		}
+		else {
+			throw new UnsupportedOperationException("Unrecognized key type.");
+		}
+
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	@Combinable
+	public static final class AggregatingUdf<T extends Product> extends RichGroupReduceFunction<T, T> {
+		private static final long serialVersionUID = 1L;
+
+		private final int[] fieldPositions;
+
+		private final AggregationFunction<Object>[] aggFunctions;
+
+		private final TypeSerializerFactory<T> serializerFactory;
+
+		private transient TupleSerializerBase<T> serializer;
+
+		public AggregatingUdf(TypeSerializerFactory<T> serializerFactory, AggregationFunction<Object>[] aggFunctions, int[] fieldPositions) {
+			Validate.notNull(serializerFactory);
+			Validate.notNull(aggFunctions);
+			Validate.isTrue(aggFunctions.length == fieldPositions.length);
+			Validate.isTrue(serializerFactory.getSerializer() instanceof TupleSerializerBase);
+
+			this.serializerFactory = serializerFactory;
+			this.aggFunctions = aggFunctions;
+			this.fieldPositions = fieldPositions;
+		}
+
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			for (int i = 0; i < aggFunctions.length; i++) {
+				aggFunctions[i].initializeAggregate();
+			}
+			serializer = (TupleSerializerBase<T>)serializerFactory.getSerializer();
+		}
+
+		@Override
+		public void reduce(Iterable<T> records, Collector<T> out) {
+			final AggregationFunction<Object>[] aggFunctions = this.aggFunctions;
+			final int[] fieldPositions = this.fieldPositions;
+
+			// aggregators are initialized from before
+
+			T current = null;
+			final Iterator<T> values = records.iterator();
+			while (values.hasNext()) {
+				current = values.next();
+
+				for (int i = 0; i < fieldPositions.length; i++) {
+					Object val = current.productElement(fieldPositions[i]);
+					aggFunctions[i].aggregate(val);
+				}
+			}
+
+			Object[] fields = new Object[serializer.getArity()];
+			// First copy all tuple fields, then overwrite the aggregated ones
+			for (int i = 0; i < fieldPositions.length; i++) {
+				fields[0] = current.productElement(i);
+			}
+			for (int i = 0; i < fieldPositions.length; i++) {
+				Object aggVal = aggFunctions[i].getAggregate();
+				fields[fieldPositions[i]] = aggVal;
+				aggFunctions[i].initializeAggregate();
+			}
+
+			T result = serializer.createInstance(fields);
+
+			out.collect(result);
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java
new file mode 100644
index 0000000..f563a8f
--- /dev/null
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.operators;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.flink.api.common.io.GenericCsvInputFormat;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
+import org.apache.flink.api.java.typeutils.runtime.RuntimeStatefulSerializerFactory;
+import org.apache.flink.api.java.typeutils.runtime.RuntimeStatelessSerializerFactory;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.types.TypeInformation;
+import org.apache.flink.types.parser.FieldParser;
+import org.apache.flink.util.StringUtils;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import scala.Product;
+
+public class ScalaCsvInputFormat<OUT extends Product> extends GenericCsvInputFormat<OUT> {
+
+	private static final long serialVersionUID = 1L;
+	
+	private transient Object[] parsedValues;
+	
+	// To speed up readRecord processing. Used to find windows line endings.
+	// It is set when open so that readRecord does not have to evaluate it
+	private boolean lineDelimiterIsLinebreak = false;
+
+	private final TypeSerializerFactory<OUT> serializerFactory;
+
+	private transient TupleSerializerBase<OUT> serializer;
+	
+	public ScalaCsvInputFormat(Path filePath, TypeInformation<OUT> typeInfo) {
+		super(filePath);
+
+		TypeSerializer<OUT> serializer = typeInfo.createSerializer();
+		if (serializer.isStateful()) {
+			serializerFactory = new RuntimeStatefulSerializerFactory<OUT>(
+					serializer, typeInfo.getTypeClass());
+		} else {
+			serializerFactory = new RuntimeStatelessSerializerFactory<OUT>(
+					serializer, typeInfo.getTypeClass());
+		}
+
+		if (!(typeInfo.isTupleType())) {
+			throw new UnsupportedOperationException("This only works on tuple types.");
+		}
+		TupleTypeInfoBase<OUT> tupleType = (TupleTypeInfoBase<OUT>) typeInfo;
+		Class[] classes = new Class[tupleType.getArity()];
+		for (int i = 0; i < tupleType.getArity(); i++) {
+			classes[i] = tupleType.getTypeAt(i).getTypeClass();
+		}
+		setFieldTypes(classes);
+	}
+
+	public void setFieldTypes(Class<?>[] fieldTypes) {
+		if (fieldTypes == null || fieldTypes.length == 0) {
+			throw new IllegalArgumentException("Field types must not be null or empty.");
+		}
+
+		setFieldTypesGeneric(fieldTypes);
+	}
+
+	public void setFields(int[] sourceFieldIndices, Class<?>[] fieldTypes) {
+		Preconditions.checkNotNull(sourceFieldIndices);
+		Preconditions.checkNotNull(fieldTypes);
+
+		checkForMonotonousOrder(sourceFieldIndices, fieldTypes);
+
+		setFieldsGeneric(sourceFieldIndices, fieldTypes);
+	}
+	
+	@Override
+	public void open(FileInputSplit split) throws IOException {
+		super.open(split);
+		
+		@SuppressWarnings("unchecked")
+		FieldParser<Object>[] fieldParsers = (FieldParser<Object>[]) getFieldParsers();
+		
+		//throw exception if no field parsers are available
+		if (fieldParsers.length == 0) {
+			throw new IOException("CsvInputFormat.open(FileInputSplit split) - no field parsers to parse input");
+		}
+		
+		// create the value holders
+		this.parsedValues = new Object[fieldParsers.length];
+		for (int i = 0; i < fieldParsers.length; i++) {
+			this.parsedValues[i] = fieldParsers[i].createValue();
+		}
+		
+		// left to right evaluation makes access [0] okay
+		// this marker is used to fasten up readRecord, so that it doesn't have to check each call if the line ending is set to default
+		if (this.getDelimiter().length == 1 && this.getDelimiter()[0] == '\n' ) {
+			this.lineDelimiterIsLinebreak = true;
+		}
+
+		serializer = (TupleSerializerBase<OUT>)serializerFactory.getSerializer();
+	}
+
+	@Override
+	public OUT readRecord(OUT reuse, byte[] bytes, int offset, int numBytes) {
+		/*
+		 * Fix to support windows line endings in CSVInputFiles with standard delimiter setup = \n
+		 */
+		//Find windows end line, so find carriage return before the newline 
+		if (this.lineDelimiterIsLinebreak == true && numBytes > 0 && bytes[offset + numBytes -1] == '\r' ) {
+			//reduce the number of bytes so that the Carriage return is not taken as data
+			numBytes--;
+		}
+		
+		if (parseRecord(parsedValues, bytes, offset, numBytes)) {
+			OUT result = serializer.createInstance(parsedValues);
+			return result;
+		} else {
+			return null;
+		}
+	}
+	
+	
+	@Override
+	public String toString() {
+		return "CSV Input (" + StringUtils.showControlCharacters(String.valueOf(getFieldDelimiter())) + ") " + getFilePath();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@SuppressWarnings("unused")
+	private static void checkAndCoSort(int[] positions, Class<?>[] types) {
+		if (positions.length != types.length) {
+			throw new IllegalArgumentException("The positions and types must be of the same length");
+		}
+		
+		TreeMap<Integer, Class<?>> map = new TreeMap<Integer, Class<?>>();
+		
+		for (int i = 0; i < positions.length; i++) {
+			if (positions[i] < 0) {
+				throw new IllegalArgumentException("The field " + " (" + positions[i] + ") is invalid.");
+			}
+			if (types[i] == null) {
+				throw new IllegalArgumentException("The type " + i + " is invalid (null)");
+			}
+			
+			if (map.containsKey(positions[i])) {
+				throw new IllegalArgumentException("The position " + positions[i] + " occurs multiple times.");
+			}
+			
+			map.put(positions[i], types[i]);
+		}
+		
+		int i = 0;
+		for (Map.Entry<Integer, Class<?>> entry : map.entrySet()) {
+			positions[i] = entry.getKey();
+			types[i] = entry.getValue();
+			i++;
+		}
+	}
+	
+	private static void checkForMonotonousOrder(int[] positions, Class<?>[] types) {
+		if (positions.length != types.length) {
+			throw new IllegalArgumentException("The positions and types must be of the same length");
+		}
+		
+		int lastPos = -1;
+		
+		for (int i = 0; i < positions.length; i++) {
+			if (positions[i] < 0) {
+				throw new IllegalArgumentException("The field " + " (" + positions[i] + ") is invalid.");
+			}
+			if (types[i] == null) {
+				throw new IllegalArgumentException("The type " + i + " is invalid (null)");
+			}
+			
+			if (positions[i] <= lastPos) {
+				throw new IllegalArgumentException("The positions must be strictly increasing (no permutations are supported).");
+			}
+			
+			lastPos = positions[i];
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java
new file mode 100644
index 0000000..07fb3a2
--- /dev/null
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.api.scala.operators;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.io.FileOutputFormat;
+import org.apache.flink.api.java.io.CsvInputFormat;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.types.TypeInformation;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import scala.Product;
+
+/**
+ * This is an OutputFormat to serialize Scala Tuples to text. The output is
+ * structured by record delimiters and field delimiters as common in CSV files.
+ * Record delimiter separate records from each other ('\n' is common). Field
+ * delimiters separate fields within a record.
+ */
+public class ScalaCsvOutputFormat<T extends Product> extends FileOutputFormat<T> implements InputTypeConfigurable {
+	private static final long serialVersionUID = 1L;
+
+	@SuppressWarnings("unused")
+	private static final Log LOG = LogFactory.getLog(ScalaCsvOutputFormat.class);
+
+	// --------------------------------------------------------------------------------------------
+
+	public static final String DEFAULT_LINE_DELIMITER = CsvInputFormat.DEFAULT_LINE_DELIMITER;
+
+	public static final String DEFAULT_FIELD_DELIMITER = String.valueOf(CsvInputFormat.DEFAULT_FIELD_DELIMITER);
+
+	// --------------------------------------------------------------------------------------------
+
+	private transient Writer wrt;
+
+	private String fieldDelimiter;
+
+	private String recordDelimiter;
+
+	private String charsetName;
+
+	private boolean allowNullValues = true;
+
+	private boolean quoteStrings = false;
+
+	// --------------------------------------------------------------------------------------------
+	// Constructors and getters/setters for the configurable parameters
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Creates an instance of CsvOutputFormat. Lines are separated by the newline character '\n',
+	 * fields are separated by ','.
+	 *
+	 * @param outputPath The path where the CSV file is written.
+	 */
+	public ScalaCsvOutputFormat(Path outputPath) {
+		this(outputPath, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER);
+	}
+
+	/**
+	 * Creates an instance of CsvOutputFormat. Lines are separated by the newline character '\n',
+	 * fields by the given field delimiter.
+	 *
+	 * @param outputPath The path where the CSV file is written.
+	 * @param fieldDelimiter
+	 *            The delimiter that is used to separate fields in a tuple.
+	 */
+	public ScalaCsvOutputFormat(Path outputPath, String fieldDelimiter) {
+		this(outputPath, DEFAULT_LINE_DELIMITER, fieldDelimiter);
+	}
+
+	/**
+	 * Creates an instance of CsvOutputFormat.
+	 *
+	 * @param outputPath The path where the CSV file is written.
+	 * @param recordDelimiter
+	 *            The delimiter that is used to separate the tuples.
+	 * @param fieldDelimiter
+	 *            The delimiter that is used to separate fields in a tuple.
+	 */
+	public ScalaCsvOutputFormat(Path outputPath, String recordDelimiter, String fieldDelimiter) {
+		super(outputPath);
+		if (recordDelimiter == null) {
+			throw new IllegalArgumentException("RecordDelmiter shall not be null.");
+		}
+
+		if (fieldDelimiter == null) {
+			throw new IllegalArgumentException("FieldDelimiter shall not be null.");
+		}
+
+		this.fieldDelimiter = fieldDelimiter;
+		this.recordDelimiter = recordDelimiter;
+		this.allowNullValues = false;
+	}
+
+	/**
+	 * Configures the format to either allow null values (writing an empty field),
+	 * or to throw an exception when encountering a null field.
+	 * <p>
+	 * by default, null values are allowed.
+	 *
+	 * @param allowNulls Flag to indicate whether the output format should accept null values.
+	 */
+	public void setAllowNullValues(boolean allowNulls) {
+		this.allowNullValues = allowNulls;
+	}
+
+	/**
+	 * Sets the charset with which the CSV strings are written to the file.
+	 * If not specified, the output format uses the systems default character encoding.
+	 *
+	 * @param charsetName The name of charset to use for encoding the output.
+	 */
+	public void setCharsetName(String charsetName) {
+		this.charsetName = charsetName;
+	}
+
+	/**
+	 * Configures whether the output format should quote string values. String values are fields
+	 * of type {@link String} and {@link org.apache.flink.types.StringValue}, as well as
+	 * all subclasses of the latter.
+	 * <p>
+	 * By default, strings are not quoted.
+	 *
+	 * @param quoteStrings Flag indicating whether string fields should be quoted.
+	 */
+	public void setQuoteStrings(boolean quoteStrings) {
+		this.quoteStrings = quoteStrings;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		this.wrt = this.charsetName == null ? new OutputStreamWriter(new BufferedOutputStream(this.stream, 4096)) :
+				new OutputStreamWriter(new BufferedOutputStream(this.stream, 4096), this.charsetName);
+	}
+
+	@Override
+	public void close() throws IOException {
+		if (wrt != null) {
+			this.wrt.close();
+		}
+		super.close();
+	}
+
+	@Override
+	public void writeRecord(T element) throws IOException {
+		int numFields = element.productArity();
+
+		for (int i = 0; i < numFields; i++) {
+			Object v = element.productElement(i);
+			if (v != null) {
+				if (i != 0) {
+					this.wrt.write(this.fieldDelimiter);
+				}
+
+				if (quoteStrings) {
+					if (v instanceof String || v instanceof StringValue) {
+						this.wrt.write('"');
+						this.wrt.write(v.toString());
+						this.wrt.write('"');
+					} else {
+						this.wrt.write(v.toString());
+					}
+				} else {
+					this.wrt.write(v.toString());
+				}
+			} else {
+				if (this.allowNullValues) {
+					if (i != 0) {
+						this.wrt.write(this.fieldDelimiter);
+					}
+				} else {
+					throw new RuntimeException("Cannot write tuple with <null> value at position: " + i);
+				}
+			}
+		}
+
+		// add the record delimiter
+		this.wrt.write(this.recordDelimiter);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	@Override
+	public String toString() {
+		return "CsvOutputFormat (path: " + this.getOutputFilePath() + ", delimiter: " + this.fieldDelimiter + ")";
+	}
+
+    /**
+	 *
+	 * The purpose of this method is solely to check whether the data type to be processed
+	 * is in fact a tuple type.
+	 */
+	@Override
+	public void setInputType(TypeInformation<?> type) {
+		if (!type.isTupleType()) {
+			throw new InvalidProgramException("The " + ScalaCsvOutputFormat.class.getSimpleName() +
+				" can only be used to write tuple data sets.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/AnnotationUtil.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/AnnotationUtil.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/AnnotationUtil.scala
deleted file mode 100644
index d23e94d..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/AnnotationUtil.scala
+++ /dev/null
@@ -1,219 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import collection.JavaConversions.asScalaIterator
-
-import org.apache.flink.api.common.operators.util.FieldSet
-import org.apache.flink.api.common.operators._
-import org.apache.flink.api.common.operators.base.{GroupReduceOperatorBase, DeltaIterationBase, BulkIterationBase, GenericDataSourceBase}
-import org.apache.flink.api.java.record.functions.FunctionAnnotation
-import org.apache.flink.api.java.record.operators.BulkIteration.PartialSolutionPlaceHolder
-import org.apache.flink.api.java.record.operators.DeltaIteration.{WorksetPlaceHolder, SolutionSetPlaceHolder}
-import org.apache.flink.api.java.record.operators.GenericDataSink
-import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsExcept
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirstExcept
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecond
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsSecondExcept
-import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable
-
-object AnnotationUtil {
-  val visited = collection.mutable.Set[Operator[_]]()
-
-  def setAnnotations(sinks: Seq[ScalaSink[_]]): Seq[ScalaSink[_]] = {
-    visited.clear()
-
-    sinks foreach setAnnotations
-
-    sinks
-  }
-
-  def setAnnotations(sink: ScalaSink[_]):Unit = {
-    setAnnotations(sink.sink.getInput)
-  }
-
-  def setAnnotations(operator: Operator[_]):Unit = {
-    if(operator != null && !visited.contains(operator)){
-      visited.add(operator)
-
-      operator match {
-        case op: GenericDataSourceBase[_,_] =>
-        case op: GenericDataSink =>
-          setAnnotations(op.getInput)
-        case op: PartialSolutionPlaceHolder =>
-        case op: SolutionSetPlaceHolder =>
-        case op: WorksetPlaceHolder =>
-        case op: DeltaIterationBase[_, _] =>
-          updateDualSemanticProperties(op)
-          setAnnotations(op.getSolutionSetDelta)
-          setAnnotations(op.getNextWorkset)
-          setAnnotations(op.getInitialWorkset)
-          setAnnotations(op.getInitialSolutionSet)
-        case op: DualInputOperator[_, _, _, _] =>
-          updateDualSemanticProperties(op)
-          setAnnotations(op.getFirstInput)
-          setAnnotations(op.getSecondInput)
-        case op: BulkIterationBase[_] =>
-          updateSingleSemanticProperties(op)
-          setAnnotations(op.getInput)
-          setAnnotations(op.getNextPartialSolution)
-          setAnnotations(op.getTerminationCriterion)
-        case op: GroupReduceOperatorBase[_, _, _] =>
-          updateCombinable(op)
-          setAnnotations(op.getInput)
-        case op: SingleInputOperator[_, _, _] =>
-          updateSingleSemanticProperties(op)
-          setAnnotations(op.getInput)
-      }
-    }
-  }
-
-  def updateCombinable(op: GroupReduceOperatorBase[_, _, _]){
-    if(op.isInstanceOf[ScalaOperator[_,_]]) {
-      val scalaOp = op.asInstanceOf[ScalaOperator[_, _]]
-
-      val combinableAnnotaion = scalaOp.getUserCodeAnnotation(classOf[Combinable])
-
-      if (combinableAnnotaion != null) {
-        op.setCombinable(true)
-      }
-    }
-  }
-
-  def updateDualSemanticProperties(op: DualInputOperator[_, _, _, _]){
-    if(op.isInstanceOf[ScalaOperator[_,_]]) {
-      val scalaOp = op.asInstanceOf[ScalaOperator[_, _]]
-      val properties = op.getSemanticProperties
-
-      // get readSet annotation from stub
-      val constantSet1Annotation: FunctionAnnotation.ConstantFieldsFirst = scalaOp.getUserCodeAnnotation(
-        classOf[FunctionAnnotation.ConstantFieldsFirst])
-      val constantSet2Annotation: FunctionAnnotation.ConstantFieldsSecond = scalaOp.getUserCodeAnnotation(
-        classOf[FunctionAnnotation.ConstantFieldsSecond])
-
-      // get readSet annotation from stub
-      val notConstantSet1Annotation: FunctionAnnotation.ConstantFieldsFirstExcept = scalaOp.getUserCodeAnnotation(
-        classOf[FunctionAnnotation.ConstantFieldsFirstExcept])
-      val notConstantSet2Annotation: FunctionAnnotation.ConstantFieldsSecondExcept = scalaOp.getUserCodeAnnotation(
-        classOf[FunctionAnnotation.ConstantFieldsSecondExcept])
-
-      if (notConstantSet1Annotation != null && constantSet1Annotation != null) {
-        throw new RuntimeException("Either ConstantFieldsFirst or ConstantFieldsFirstExcept can be specified, not both.")
-      }
-
-      if (constantSet2Annotation != null && notConstantSet2Annotation != null) {
-        throw new RuntimeException("Either ConstantFieldsSecond or ConstantFieldsSecondExcept can be specified, not both.")
-      }
-
-      // extract readSets from annotations
-      if (notConstantSet1Annotation != null) {
-        for (element <- notConstantSet1Annotation.value()) {
-          if (properties.getForwardedField1(element) != null) {
-            throw new RuntimeException("Field " + element + " cannot be forwarded and non constant at the same time.")
-          }
-        }
-
-        val fieldSet = new FieldSet(notConstantSet1Annotation.value(): _*)
-
-        for (i <- 0 until scalaOp.getUDF.getOutputLength) {
-          if (!fieldSet.contains(i)) {
-            properties.addForwardedField1(i, i)
-          }
-        }
-      } else if (constantSet1Annotation != null) {
-        for (value <- constantSet1Annotation.value) {
-          properties.addForwardedField1(value, value)
-        }
-      }
-
-      if (notConstantSet2Annotation != null) {
-        for (element <- notConstantSet2Annotation.value()) {
-          if (properties.getForwardedField2(element) != null) {
-            throw new RuntimeException("Field " + element + " cannot be forwarded and non constant at the same time.")
-          }
-        }
-
-        val fieldSet = new FieldSet(notConstantSet2Annotation.value(): _*)
-
-        for (i <- 0 until scalaOp.getUDF.getOutputLength) {
-          if (!fieldSet.contains(i)) {
-            properties.addForwardedField2(i, i)
-          }
-        }
-      } else if (constantSet2Annotation != null) {
-        for (value <- constantSet2Annotation.value) {
-          properties.addForwardedField2(value, value)
-        }
-      }
-
-      op.setSemanticProperties(properties)
-    }
-  }
-
-  def updateSingleSemanticProperties(op: SingleInputOperator[_, _, _]) {
-    if (op.isInstanceOf[ScalaOperator[_, _]]) {
-      val scalaOp = op.asInstanceOf[ScalaOperator[_, _]]
-      var properties = op.getSemanticProperties
-
-      if (properties == null) {
-        properties = new SingleInputSemanticProperties()
-      }
-
-      // get constantSet annotation from stub
-      val constantSet: FunctionAnnotation.ConstantFields =
-        scalaOp.getUserCodeAnnotation(classOf[FunctionAnnotation.ConstantFields])
-      val notConstantSet: FunctionAnnotation.ConstantFieldsExcept =
-        scalaOp.getUserCodeAnnotation(classOf[FunctionAnnotation.ConstantFieldsExcept])
-
-      if (notConstantSet != null && constantSet != null) {
-        throw new RuntimeException("Either ConstantFields or ConstantFieldsExcept can be specified, not both.")
-      }
-
-      // extract notConstantSet from annotation
-      if (notConstantSet != null) {
-        val nonConstant: FieldSet = new FieldSet(notConstantSet.value: _*)
-
-        for (element <- nonConstant.iterator()) {
-          if (properties.getForwardedField(element) != null) {
-            throw new RuntimeException("Field " + element + " is non constant and at the same time forwarded. This " +
-              "cannot happen.")
-          }
-        }
-
-        for (i <- 0 until scalaOp.getUDF.getOutputLength) {
-          if (!nonConstant.contains(i)) {
-            properties.addForwardedField(i, i)
-          }
-        }
-
-      } else if (constantSet != null) {
-        // extract constantSet from annotation
-        for (value <- constantSet.value) {
-          properties.addForwardedField(value, value)
-        }
-      }
-
-      op.setSemanticProperties(properties)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/CompilerHints.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/CompilerHints.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/CompilerHints.scala
deleted file mode 100644
index a03f5cf..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/CompilerHints.scala
+++ /dev/null
@@ -1,386 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala
-
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import scala.util.DynamicVariable
-
-import org.apache.flink.api.scala.codegen.MacroContextHolder
-import org.apache.flink.api.scala.analysis._
-import org.apache.flink.api.scala.analysis.FieldSet.toSeq
-import org.apache.flink.api.scala.analysis.UDF2
-import org.apache.flink.api.scala.analysis.UDF1
-import org.apache.flink.api.scala.analysis.FieldSelector
-
-import org.apache.flink.api.common.operators.Operator
-import org.apache.flink.api.common.operators.util.{FieldSet => PactFieldSet}
-import org.apache.flink.types.Record
-
-
-case class KeyCardinality(
-    key: FieldSelector,
-    isUnique: Boolean,
-    distinctCount: Option[Long],
-    avgNumRecords: Option[Float]) {
-
-  @transient private var pactFieldSets =
-    collection.mutable.Map[Operator[Record] with ScalaOperator[_, _], PactFieldSet]()
-
-  def getPactFieldSet(contract: Operator[Record] with ScalaOperator[_, _]): PactFieldSet = {
-
-    if (pactFieldSets == null) {
-      pactFieldSets =
-        collection.mutable.Map[Operator[Record] with ScalaOperator[_, _], PactFieldSet]()
-    }
-
-    val keyCopy = key.copy()
-    contract.getUDF.attachOutputsToInputs(keyCopy.inputFields)
-    val keySet = keyCopy.selectedFields.toIndexSet.toArray
-
-    val fieldSet = pactFieldSets.getOrElseUpdate(contract, new PactFieldSet(keySet, true))
-    fieldSet
-  }
-}
-
-trait OutputHintable[Out] { this: DataSet[Out] =>
-  def getContract = contract
-  
-  private var _cardinalities: List[KeyCardinality] = List[KeyCardinality]()
-  
-  def addCardinality(card: KeyCardinality) {
-    _cardinalities = card :: _cardinalities
-    applyHints(getContract)
-  }
-
-  def degreeOfParallelism = contract.getDegreeOfParallelism()
-  def degreeOfParallelism_=(value: Int) = contract.setDegreeOfParallelism(value)
-  def degreeOfParallelism(value: Int): this.type = { contract.setDegreeOfParallelism(value); this }
-    
-  def outputSize = contract.getCompilerHints().getOutputSize()
-  def outputSize_=(value: Long) = contract.getCompilerHints().setOutputSize(value)
-  def outputSize(value: Long): this.type = {
-    contract.getCompilerHints().setOutputSize(value)
-    this
-  }
-  
-  def outputCardinality = contract.getCompilerHints().getOutputCardinality()
-  def outputCardinality_=(value: Long) = contract.getCompilerHints().setOutputCardinality(value)
-  def outputCardinality(value: Long): this.type = {
-    contract.getCompilerHints().setOutputCardinality(value)
-    this
-  }
-  
-  def avgBytesPerRecord = contract.getCompilerHints().getAvgOutputRecordSize()
-  def avgBytesPerRecord_=(value: Float) = contract.getCompilerHints().setAvgOutputRecordSize(value)
-  def avgBytesPerRecord(value: Float): this.type = {
-    contract.getCompilerHints().setAvgOutputRecordSize(value)
-    this
-  }
-
-  def filterFactor = contract.getCompilerHints().getFilterFactor()
-  def filterFactor_=(value: Float) = contract.getCompilerHints().setFilterFactor(value)
-  def filterFactor(value: Float): this.type = {
-    contract.getCompilerHints().setFilterFactor(value)
-    this
-  }
-
-  def uniqueKey[Key](fields: Out => Key) = macro OutputHintableMacros.uniqueKey[Out, Key]
-
-  def applyHints(contract: Operator[Record] with ScalaOperator[_, _]): Unit = {
-    val hints = contract.getCompilerHints
-
-    if (hints.getUniqueFields != null) {
-      hints.getUniqueFields.clear()
-    }
-
-    _cardinalities.foreach { card =>
-
-      val fieldSet = card.getPactFieldSet(contract)
-
-      if (card.isUnique) {
-        hints.addUniqueField(fieldSet)
-      }
-    }
-  }
-}
-
-object OutputHintableMacros {
-  
-  def uniqueKey[Out: c.WeakTypeTag, Key: c.WeakTypeTag]
-      (c: Context { type PrefixType = OutputHintable[Out] })(fields: c.Expr[Out => Key])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedKeySelector = slave.getSelector(fields)
-
-    val result = reify {
-      val contract = c.prefix.splice.getContract
-      val hints = contract.getCompilerHints
-      
-      val keySelection = generatedKeySelector.splice
-      val key = new FieldSelector(c.prefix.splice.getContract.getUDF.outputUDT, keySelection)
-      val card = KeyCardinality(key, true, None, None)
-      
-      c.prefix.splice.addCardinality(card)
-    }
-    result
-  }
-  
-  def uniqueKeyWithDistinctCount[Out: c.WeakTypeTag, Key: c.WeakTypeTag]
-      (c: Context { type PrefixType = OutputHintable[Out] })
-      (fields: c.Expr[Out => Key], distinctCount: c.Expr[Long])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedKeySelector = slave.getSelector(fields)
-
-    val result = reify {
-      val contract = c.prefix.splice.getContract
-      val hints = contract.getCompilerHints
-      
-      val keySelection = generatedKeySelector.splice
-      val key = new FieldSelector(c.prefix.splice.getContract.getUDF.outputUDT, keySelection)
-      val card = KeyCardinality(key, true, Some(distinctCount.splice), None)
-      
-      c.prefix.splice.addCardinality(card)
-    }
-    result
-  }
-  
-  def cardinality[Out: c.WeakTypeTag, Key: c.WeakTypeTag]
-      (c: Context { type PrefixType = OutputHintable[Out] })(fields: c.Expr[Out => Key])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedKeySelector = slave.getSelector(fields)
-
-    val result = reify {
-      val contract = c.prefix.splice.getContract
-      val hints = contract.getCompilerHints
-      
-      val keySelection = generatedKeySelector.splice
-      val key = new FieldSelector(c.prefix.splice.getContract.getUDF.outputUDT, keySelection)
-      val card = KeyCardinality(key, false, None, None)
-      
-      c.prefix.splice.addCardinality(card)
-    }
-    result
-  }
-  
-  def cardinalityWithDistinctCount[Out: c.WeakTypeTag, Key: c.WeakTypeTag]
-      (c: Context { type PrefixType = OutputHintable[Out] })
-      (fields: c.Expr[Out => Key], distinctCount: c.Expr[Long])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedKeySelector = slave.getSelector(fields)
-
-    val result = reify {
-      val contract = c.prefix.splice.getContract
-      val hints = contract.getCompilerHints
-      
-      val keySelection = generatedKeySelector.splice
-      val key = new FieldSelector(c.prefix.splice.getContract.getUDF.outputUDT, keySelection)
-      val card = KeyCardinality(key, false, Some(distinctCount.splice), None)
-      
-      c.prefix.splice.addCardinality(card)
-    }
-    result
-  }
-  
-  def cardinalityWithAvgNumRecords[Out: c.WeakTypeTag, Key: c.WeakTypeTag]
-      (c: Context { type PrefixType = OutputHintable[Out] })
-      (fields: c.Expr[Out => Key], avgNumRecords: c.Expr[Float]): c.Expr[Unit] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedKeySelector = slave.getSelector(fields)
-
-    val result = reify {
-      val contract = c.prefix.splice.getContract
-      val hints = contract.getCompilerHints
-      
-      val keySelection = generatedKeySelector.splice
-      val key = new FieldSelector(c.prefix.splice.getContract.getUDF.outputUDT, keySelection)
-      val card = KeyCardinality(key, false, None, Some(avgNumRecords.splice))
-      
-      c.prefix.splice.addCardinality(card)
-    }
-    result
-  }
-  
-  def cardinalityWithAll[Out: c.WeakTypeTag, Key: c.WeakTypeTag]
-      (c: Context { type PrefixType = OutputHintable[Out] })
-      (fields: c.Expr[Out => Key], distinctCount: c.Expr[Long], avgNumRecords: c.Expr[Float])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedKeySelector = slave.getSelector(fields)
-
-    val result = reify {
-      val contract = c.prefix.splice.getContract
-      
-      val keySelection = generatedKeySelector.splice
-      val key = new FieldSelector(contract.getUDF.outputUDT, keySelection)
-      val card = KeyCardinality(key, false, Some(distinctCount.splice), Some(avgNumRecords.splice))
-      
-      c.prefix.splice.addCardinality(card)
-    }
-    result
-  }
-}
-
-trait InputHintable[In, Out] { this: DataSet[Out] =>
-  def markUnread: Int => Unit
-  def markCopied: (Int, Int) => Unit
-  
-  def getInputUDT: UDT[In]
-  def getOutputUDT: UDT[Out]
-
-  def neglects[Fields](fields: In => Fields): Unit =
-    macro InputHintableMacros.neglects[In, Out, Fields]
-  def observes[Fields](fields: In => Fields): Unit =
-    macro InputHintableMacros.observes[In, Out, Fields]
-  def preserves[Fields](from: In => Fields, to: Out => Fields) =
-    macro InputHintableMacros.preserves[In, Out, Fields]
-}
-
-object InputHintable {
-
-  private val enabled = new DynamicVariable[Boolean](true)
-
-  def withEnabled[T](isEnabled: Boolean)(thunk: => T): T = enabled.withValue(isEnabled) { thunk }
-  
-}
-
-object InputHintableMacros {
-  
-  def neglects[In: c.WeakTypeTag, Out: c.WeakTypeTag, Fields: c.WeakTypeTag]
-      (c: Context { type PrefixType = InputHintable[In, Out] })
-      (fields: c.Expr[In => Fields])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedFieldSelector = slave.getSelector(fields)
-
-    val result = reify {
-      val fieldSelection = generatedFieldSelector.splice
-      val fieldSelector = new FieldSelector(c.prefix.splice.getInputUDT, fieldSelection)
-      val unreadFields = fieldSelector.selectedFields.map(_.localPos).toSet
-      unreadFields.foreach(c.prefix.splice.markUnread(_))
-    }
-    result
-  }
-  
-  def observes[In: c.WeakTypeTag, Out: c.WeakTypeTag, Fields: c.WeakTypeTag]
-      (c: Context { type PrefixType = InputHintable[In, Out] })
-      (fields: c.Expr[In => Fields])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-    
-    val generatedFieldSelector = slave.getSelector(fields)
-
-    val result = reify {
-      val fieldSelection = generatedFieldSelector.splice
-      val fieldSelector = new FieldSelector(c.prefix.splice.getInputUDT, fieldSelection)
-      val fieldSet = fieldSelector.selectedFields.map(_.localPos).toSet
-      val unreadFields = fieldSelector.inputFields.map(_.localPos).toSet.diff(fieldSet)
-      unreadFields.foreach(c.prefix.splice.markUnread(_))
-    }
-    result
-  }
-  
-  def preserves[In: c.WeakTypeTag, Out: c.WeakTypeTag, Fields: c.WeakTypeTag]
-      (c: Context { type PrefixType = InputHintable[In, Out] })
-      (from: c.Expr[In => Fields], to: c.Expr[Out => Fields])
-    : c.Expr[Unit] = {
-
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-
-    val generatedFromFieldSelector = slave.getSelector(from)
-    val generatedToFieldSelector = slave.getSelector(to)
-
-    val result = reify {
-      val fromSelection = generatedFromFieldSelector.splice
-      val fromSelector = new FieldSelector(c.prefix.splice.getInputUDT, fromSelection)
-      val toSelection = generatedToFieldSelector.splice
-      val toSelector = new FieldSelector(c.prefix.splice.getOutputUDT, toSelection)
-      val pairs = fromSelector.selectedFields.map(_.localPos)
-        .zip(toSelector.selectedFields.map(_.localPos))
-      pairs.foreach(c.prefix.splice.markCopied.tupled)
-    }
-    result
-  }
-}
-
-trait OneInputHintable[In, Out] extends InputHintable[In, Out] with OutputHintable[Out] {
-  this: DataSet[Out] =>
-	override def markUnread = contract.getUDF.asInstanceOf[UDF1[In, Out]].markInputFieldUnread _ 
-	override def markCopied = contract.getUDF.asInstanceOf[UDF1[In, Out]].markFieldCopied _ 
-	
-	override def getInputUDT = contract.getUDF.asInstanceOf[UDF1[In, Out]].inputUDT
-	override def getOutputUDT = contract.getUDF.asInstanceOf[UDF1[In, Out]].outputUDT
-}
-
-trait TwoInputHintable[LeftIn, RightIn, Out] extends OutputHintable[Out] { this: DataSet[Out] =>
-  val left = new DataSet[Out](contract) with OneInputHintable[LeftIn, Out] {
-	override def markUnread = { pos: Int => contract.getUDF.asInstanceOf[UDF2[LeftIn, RightIn, Out]]
-    .markInputFieldUnread(Left(pos))}
-	override def markCopied = { (from: Int, to: Int) => contract.getUDF
-    .asInstanceOf[UDF2[LeftIn, RightIn, Out]].markFieldCopied(Left(from), to)}
-	override def getInputUDT = contract.getUDF.asInstanceOf[UDF2[LeftIn, RightIn, Out]].leftInputUDT
-	override def getOutputUDT = contract.getUDF.asInstanceOf[UDF2[LeftIn, RightIn, Out]].outputUDT
-  }
-  
-  val right = new DataSet[Out](contract) with OneInputHintable[RightIn, Out] {
-	override def markUnread = { pos: Int => contract.getUDF.asInstanceOf[UDF2[LeftIn, RightIn, Out]]
-    .markInputFieldUnread(Right(pos))}
-	override def markCopied = { (from: Int, to: Int) => contract.getUDF
-    .asInstanceOf[UDF2[LeftIn, RightIn, Out]].markFieldCopied(Right(from), to)}
-	override def getInputUDT = contract.getUDF.asInstanceOf[UDF2[LeftIn, RightIn, Out]].rightInputUDT
-	override def getOutputUDT = contract.getUDF.asInstanceOf[UDF2[LeftIn, RightIn, Out]].outputUDT
-  }
-}


[16/60] git commit: Change ObjectArrayTypeInfo.getInfoFor to use componentInfo.isTupleType()

Posted by al...@apache.org.
Change ObjectArrayTypeInfo.getInfoFor to use componentInfo.isTupleType()

This is necessary because the previous test didn't work for Scala tuples.


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

Branch: refs/heads/master
Commit: 16dad15fb26ee6dd9f5300f2138c22cb767d5351
Parents: 785f2c4
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 2 18:40:17 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:57 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/16dad15f/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
index b7569ac..0802280 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
@@ -116,7 +116,7 @@ public class ObjectArrayTypeInfo<T, C> extends TypeInformation<T> {
 			return new ObjectArrayTypeInfo<T, C>(type, genericArray.getGenericComponentType(), componentInfo);
 		}
 		// for tuples without generics (e.g. generated by the TypeInformation parser)
-		else if (type instanceof Class<?> && ((Class<?>) type).isArray() && Tuple.class.isAssignableFrom(((Class<?>) type).getComponentType())
+		else if (type instanceof Class<?> && ((Class<?>) type).isArray() && componentInfo.isTupleType()
 				&& type != Tuple.class) {
 			return new ObjectArrayTypeInfo<T, C>(type, ((Class<?>) type).getComponentType(), componentInfo);
 		}


[49/60] git commit: [doc] Switch parser to kramdown, normalize Headings

Posted by al...@apache.org.
[doc] Switch parser to kramdown, normalize Headings

The switch to kramdown is necessary because I want to add tabs in the
documentation for code examples and Redcarpet does not allow markup
inside divs.

Before, some doc pages had "#" headings as toplevel headings while
others had "##" (which is the same as --- underlined headings). Now we
user level 2 everywhere. The page title is still a h1 heading.


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

Branch: refs/heads/master
Commit: 4ddc3f72561e3acc6f160fd210de2c8ab3ae50c5
Parents: ca4fa3d
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Sep 18 11:22:36 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 docs/_config.yml                   |   10 +-
 docs/_layouts/docs.html            |   10 +-
 docs/_plugins/gh_link.rb           |    2 +-
 docs/_plugins/kramdown_pygments.rb |   92 ++
 docs/_plugins/tocify.rb            |   10 -
 docs/building.md                   |   35 +-
 docs/cli.md                        |   12 +-
 docs/cluster_execution.md          |   37 +-
 docs/cluster_setup.md              |  103 ++-
 docs/config.md                     |   25 +-
 docs/css/codetabs.css              |   43 +
 docs/dataset_transformations.md    |  677 ++++++++++++++
 docs/example_connectors.md         |   51 +-
 docs/faq.md                        |   57 +-
 docs/internal_add_operator.md      |   65 +-
 docs/internal_general_arch.md      |    3 +
 docs/internal_logging.md           |   26 +-
 docs/internal_overview.md          |    4 +-
 docs/iterations.md                 |   24 +-
 docs/java_api_examples.md          |   39 +-
 docs/java_api_guide.md             | 1261 -------------------------
 docs/java_api_quickstart.md        |   33 +-
 docs/java_api_transformations.md   |  677 --------------
 docs/js/codetabs.js                |  121 +++
 docs/local_execution.md            |   26 +-
 docs/local_setup.md                |   52 +-
 docs/programming_guide.md          | 1522 +++++++++++++++++++++++++++++++
 docs/run_example_quickstart.md     |   48 +-
 docs/scala_api_quickstart.md       |   13 +-
 docs/setup_quickstart.md           |   85 +-
 docs/spargel_guide.md              |   20 +-
 docs/web_client.md                 |   11 +-
 docs/yarn_setup.md                 |   65 +-
 33 files changed, 2926 insertions(+), 2333 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 6d17a12..95841ca 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -30,9 +30,7 @@ defaults:
       layout: docs
 
 highlighter: pygments
-markdown: redcarpet
-redcarpet:
-  # https://help.github.com/articles/github-flavored-markdown
-  extensions: ["no_intra_emphasis", "fenced_code_blocks", "autolink",
-               "tables", "with_toc_data", "strikethrough", "superscript",
-               "lax_spacing"]
+markdown: KramdownPygments
+
+kramdown:
+    toc_levels: 1..3

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/docs.html b/docs/_layouts/docs.html
index 88d081e..2cf37ce 100644
--- a/docs/_layouts/docs.html
+++ b/docs/_layouts/docs.html
@@ -8,6 +8,7 @@
 
     <link rel="stylesheet" href="//netdna.bootstrapcdn.com/bootstrap/3.1.1/css/bootstrap.min.css">
     <link rel="stylesheet" href="css/syntax.css">
+    <link rel="stylesheet" href="css/codetabs.css">
 
     <!-- HTML5 Shim and Respond.js IE8 support of HTML5 elements and media queries -->
     <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
@@ -46,9 +47,8 @@
 
                     <li>Programming Guides
                         <ul>
-                            <li><a href="java_api_guide.html">Java API</a></li>
-                            <li><a href="java_api_transformations.html">Java API Transformations</a></li>
-                            <li><a href="scala_api_guide.html">Scala API</a></li>
+                            <li><a href="programming_guide.html">Flink Programming Guide</a></li>
+                            <li><a href="dataset_transformations.html">DataSet Transformations</a></li>
                             <li><a href="hadoop_compatability.html">Hadoop Compatability</a></li>
                             <li><a href="iterations.html">Iterations</a></li>
                             <li><a href="spargel_guide.html">Spargel Graph API</a></li>
@@ -77,6 +77,7 @@
                             <li><a href="internal_overview.html">Overview</a></li>
                             <li><a href="internal_general_arch.html">General Architecture</a></li>
                             <li><a href="internal_add_operator.html">How-to: Adding a new Operator</a></li>
+                            <li><a href="internal_logging.html">How-to: Logging</a></li>
                         </ul>
                     </li>
                 </ul>
@@ -84,8 +85,6 @@
             <div class="col-md-9">
                 <h1>{{ page.title }}</h1>
 
-                {{ page.content | tocify }}
-
                 {{ content }}
             </div>
         </div>
@@ -93,5 +92,6 @@
 
     <script src="https://ajax.googleapis.com/ajax/libs/jquery/1.11.0/jquery.min.js"></script>
     <script src="//netdna.bootstrapcdn.com/bootstrap/3.1.1/js/bootstrap.min.js"></script>
+    <script src="js/codetabs.js"></script>
   </body>
 </html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/_plugins/gh_link.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/gh_link.rb b/docs/_plugins/gh_link.rb
index 3b51eb2..58dca2a 100644
--- a/docs/_plugins/gh_link.rb
+++ b/docs/_plugins/gh_link.rb
@@ -23,7 +23,7 @@ module Jekyll
       name = name.to_s == '' ? file : name
       #refname = input[2].nil? ? file : input[2]
 
-      "<a href=#{config["FLINK_GITHUB_URL"]}/blob/#{gh_tag}/#{path}>#{name}</a>"
+      "[#{name}](#{config["FLINK_GITHUB_URL"]}/blob/#{gh_tag}/#{path})"
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/_plugins/kramdown_pygments.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/kramdown_pygments.rb b/docs/_plugins/kramdown_pygments.rb
new file mode 100644
index 0000000..89bb911
--- /dev/null
+++ b/docs/_plugins/kramdown_pygments.rb
@@ -0,0 +1,92 @@
+# We define the an additional option for the kramdown parser to look for
+module Kramdown
+  module Options
+      define(:kramdown_default_lang, Symbol, nil, <<EOF)
+Sets the default language for highlighting code blocks
+
+If no language is set for a code block, the default language is used
+instead. The value has to be one of the languages supported by pygments
+or nil if no default language should be used.
+
+Default: nil
+Used by: PygmentsHtml converter
+EOF
+  end
+end
+
+# This class is a plugin for kramdown, to make it use pygments instead of coderay
+# It has nothing to do with Jekyll, it is simply used by the custom converter below
+module Kramdown
+  module Converter
+    class PygmentsHtml < Html
+
+      begin
+        require 'pygments'
+      rescue LoadError
+        STDERR.puts 'You are missing a library required for syntax highlighting. Please run:'
+        STDERR.puts '  $ [sudo] gem install pygments'
+        raise FatalException.new("Missing dependency: Pygments")
+      end
+
+      def convert_codeblock(el, indent)
+        attr = el.attr.dup
+        lang = extract_code_language!(attr) || @options[:kramdown_default_lang]
+        code = pygmentize(el.value, lang)
+        code_attr = {}
+        code_attr['class'] = "language-#{lang}" if lang
+        "#{' '*indent}<div class=\"highlight\"><pre#{html_attributes(attr)}><code#{html_attributes(code_attr)}>#{code}</code></pre></div>\n"
+      end
+
+      def convert_codespan(el, indent)
+        attr = el.attr.dup
+        lang = extract_code_language!(attr) || @options[:kramdown_default_lang]
+        code = pygmentize(el.value, lang)
+        if lang
+          attr['class'] = "highlight"
+          if attr.has_key?('class')
+            attr['class'] += " language-#{lang}"
+          else
+            attr['class'] = "language-#{lang}"
+          end
+        end
+        "<code#{html_attributes(attr)}>#{code}</code>"
+      end
+      
+      def pygmentize(code, lang)
+        if lang
+          Pygments.highlight(code,
+            :lexer => lang,
+            :options => { :startinline => true, :encoding => 'utf-8', :nowrap => true })
+        else
+          escape_html(code)
+        end
+      end
+    end
+  end
+end
+
+# This class is the actual custom Jekyll converter.
+class Jekyll::Converters::Markdown::KramdownPygments
+
+  def initialize(config)
+    require 'kramdown'
+    @config = config
+  rescue LoadError
+    STDERR.puts 'You are missing a library required for Markdown. Please run:'
+    STDERR.puts '  $ [sudo] gem install kramdown'
+    raise FatalException.new("Missing dependency: kramdown")
+  end
+
+  def convert(content)
+    html = Kramdown::Document.new(content, {
+        :auto_ids             => @config['kramdown']['auto_ids'],
+        :footnote_nr          => @config['kramdown']['footnote_nr'],
+        :entity_output        => @config['kramdown']['entity_output'],
+        :toc_levels           => @config['kramdown']['toc_levels'],
+        :smart_quotes         => @config['kramdown']['smart_quotes'],
+        :kramdown_default_lang => @config['kramdown']['default_lang'],
+        :input                => @config['kramdown']['input']
+    }).to_pygments_html
+    return html;
+  end
+end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/_plugins/tocify.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/tocify.rb b/docs/_plugins/tocify.rb
deleted file mode 100644
index 9eb6293..0000000
--- a/docs/_plugins/tocify.rb
+++ /dev/null
@@ -1,10 +0,0 @@
-module Jekyll
-  module Tocify
-    def tocify(input)
-      converter = Redcarpet::Markdown.new(Redcarpet::Render::HTML_TOC.new(nesting_level: 2))
-      converter.render(input)
-    end
-  end
-end
-
-Liquid::Template.register_filter(Jekyll::Tocify)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/building.md
----------------------------------------------------------------------
diff --git a/docs/building.md b/docs/building.md
index 842dbc1..544dfba 100644
--- a/docs/building.md
+++ b/docs/building.md
@@ -2,20 +2,20 @@
 title:  "Build Flink"
 ---
 
-
 In order to build Flink, you need the source code. Either download the source of a release or clone the git repository. In addition to that, you need Maven 3 and a JDK (Java Development Kit). Note that you can not build Flink with Oracle JDK 6 due to a unresolved bug in the Oracle Java compiler. It works well with OpenJDK 6 and all Java 7 and 8 compilers.
 
 To clone from git, enter:
-```
+
+~~~bash
 git clone {{ site.FLINK_GITHUB_URL }}
-```
+~~~
 
 The simplest way of building Flink is by running:
 
-```
+~~~bash
 cd incubator-flink
 mvn clean package -DskipTests
-```
+~~~
 
 This instructs Maven (`mvn`) to first remove all existing builds (`clean`) and then create a new Flink binary (`package`). The `-DskipTests` command prevents Maven from executing the unit tests.
 
@@ -27,12 +27,12 @@ This section covers building Flink for a specific Hadoop version. Most users do
 
 The problem is that Flink uses HDFS and YARN which are both dependencies from Apache Hadoop. There exist many different versions of Hadoop (from both the upstream project and the different Hadoop distributions). If a user is using a wrong combination of versions, exceptions like this one occur:
 
-```
+~~~bash
 ERROR: The job was not successfully submitted to the nephele job manager:
     org.apache.flink.nephele.executiongraph.GraphConversionException: Cannot compute input splits for TSV:
     java.io.IOException: Failed on local exception: com.google.protobuf.InvalidProtocolBufferException:
     Protocol message contained an invalid tag (zero).; Host Details :
-```
+~~~
 
 There are two main versions of Hadoop that we need to differentiate:
 - Hadoop 1, with all versions starting with zero or one, like 0.20, 0.23 or 1.2.1.
@@ -40,23 +40,26 @@ There are two main versions of Hadoop that we need to differentiate:
 The main differentiation between Hadoop 1 and Hadoop 2 is the availability of Hadoop YARN (Hadoops cluster resource manager).
 
 **To build Flink for Hadoop 2**, issue the following command:
-```
+
+~~~bash
 mvn clean package -DskipTests -Dhadoop.profile=2
-```
+~~~
 
 The `-Dhadoop.profile=2` flag instructs Maven to build Flink with YARN support and the Hadoop 2 HDFS client.
 
 Usually, this flag is sufficient for full support of Flink for Hadoop 2-versions.
 However, you can also **specify a specific Hadoop version to build against**:
-```
+
+~~~bash
 mvn clean package -DskipTests -Dhadoop.profile=2 -Dhadoop.version=2.4.1
-```
+~~~
 
 
 **To build Flink against a vendor specific Hadoop version**, issue the following command:
-```
+
+~~~bash
 mvn clean package -DskipTests -Pvendor-repos -Dhadoop.profile=2 -Dhadoop.version=2.2.0-cdh5.0.0-beta-2
-```
+~~~
 
 The `-Pvendor-repos` activates a Maven [build profile](http://maven.apache.org/guides/introduction/introduction-to-profiles.html) that includes the repositories of popular Hadoop vendors such as Cloudera, Hortonworks, or MapR.
 
@@ -66,11 +69,11 @@ Maven will automatically build Flink with its YARN client if the `-Dhadoop.profi
 
 So if you are building Flink for Hadoop `2.0.0-alpha`, use the following command:
 
-```bash
+~~~bash
 -P\!include-yarn -Dhadoop.profile=2 -Dhadoop.version=2.0.0-alpha
-```
+~~~
 
-### Background
+## Background
 
 The builds with Maven are controlled by [properties](http://maven.apache.org/pom.html#Properties) and <a href="http://maven.apache.org/guides/introduction/introduction-to-profiles.html">build profiles</a>.
 There are two profiles, one for hadoop1 and one for hadoop2. When the hadoop2 profile is enabled, the system will also build the YARN client.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/cli.md
----------------------------------------------------------------------
diff --git a/docs/cli.md b/docs/cli.md
index f716065..0c31224 100644
--- a/docs/cli.md
+++ b/docs/cli.md
@@ -2,6 +2,10 @@
 title:  "Command-Line Interface"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
+
 Flink provides a command-line interface to run programs that are packaged
 as JAR files, and control their execution.  The command line interface is part
 of any Flink setup, available in local single node setups and in
@@ -20,7 +24,7 @@ The command line can be used to
 - provide information about a job, and
 - list running and waiting jobs.
 
-# Examples
+## Examples
 
 -   Run example program with no arguments.
 
@@ -61,11 +65,11 @@ The command line can be used to
 
         ./bin/flink cancel -i <jobID>
 
-# Usage
+## Usage
 
 The command line syntax is as follows:
 
-```
+~~~
 ./flink <ACTION> [OPTIONS] [ARGUMENTS]
 
 General options:
@@ -126,4 +130,4 @@ Action "cancel" cancels a submitted Flink program.
      -i,--jobid <jobID>             JobID of program to cancel
      
      -m,--jobmanager <host:port>    Option to connect to a different Flink master (JobManager).
-```
+~~~

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/cluster_execution.md
----------------------------------------------------------------------
diff --git a/docs/cluster_execution.md b/docs/cluster_execution.md
index 1d74f7c..d52d3f0 100644
--- a/docs/cluster_execution.md
+++ b/docs/cluster_execution.md
@@ -2,10 +2,13 @@
 title:  "Cluster Execution"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Flink programs can run distributed on clusters of many machines. There
 are two ways to send a program to a cluster for execution:
 
-# Command Line Interface
+## Command Line Interface
 
 The command line interface lets you submit packaged programs (JARs) to a cluster
 (or single machine setup).
@@ -13,30 +16,30 @@ The command line interface lets you submit packaged programs (JARs) to a cluster
 Please refer to the [Command Line Interface](cli.html) documentation for
 details.
 
-# Remote Environment
+## Remote Environment
 
 The remote environment lets you execute Flink Java programs on a cluster
 directly. The remote environment points to the cluster on which you want to
 execute the program.
 
-## Maven Dependency
+### Maven Dependency
 
 If you are developing your program as a Maven project, you have to add the
 `flink-clients` module using this dependency:
 
-```xml
+~~~xml
 <dependency>
   <groupId>org.apache.flink</groupId>
   <artifactId>flink-clients</artifactId>
   <version>{{ site.FLINK_VERSION_STABLE }}</version>
 </dependency>
-```
+~~~
 
-## Example
+### Example
 
 The following illustrates the use of the `RemoteEnvironment`:
 
-```java
+~~~java
 public static void main(String[] args) throws Exception {
     ExecutionEnvironment env = ExecutionEnvironment
         .createRemoteEnvironment("strato-master", "7661", "/home/user/udfs.jar");
@@ -53,36 +56,36 @@ public static void main(String[] args) throws Exception {
 
     env.execute();
 }
-```
+~~~
 
 Note that the program contains custom UDFs and hence requires a JAR file with
 the classes of the code attached. The constructor of the remote environment
 takes the path(s) to the JAR file(s).
 
-# Remote Executor
+## Remote Executor
 
 Similar to the RemoteEnvironment, the RemoteExecutor lets you execute
 Flink programs on a cluster directly. The remote executor accepts a
 *Plan* object, which describes the program as a single executable unit.
 
-## Maven Dependency
+### Maven Dependency
 
 If you are developing your program in a Maven project, you have to add the
 `flink-clients` module using this dependency:
 
-```xml
+~~~xml
 <dependency>
   <groupId>org.apache.flink</groupId>
   <artifactId>flink-clients</artifactId>
   <version>{{ site.FLINK_VERSION_STABLE }}</version>
 </dependency>
-```
+~~~
 
-## Example
+### Example
 
 The following illustrates the use of the `RemoteExecutor` with the Scala API:
 
-```scala
+~~~scala
 def main(args: Array[String]) {
     val input = TextFile("hdfs://path/to/file")
 
@@ -95,12 +98,12 @@ def main(args: Array[String]) {
     val executor = new RemoteExecutor("strato-master", 7881, "/path/to/jarfile.jar")
     executor.executePlan(p);
 }
-```
+~~~
 
 The following illustrates the use of the `RemoteExecutor` with the Java API (as
 an alternative to the RemoteEnvironment):
 
-```java
+~~~java
 public static void main(String[] args) throws Exception {
     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -118,7 +121,7 @@ public static void main(String[] args) throws Exception {
     RemoteExecutor e = new RemoteExecutor("strato-master", 7881, "/path/to/jarfile.jar");
     e.executePlan(p);
 }
-```
+~~~
 
 Note that the program contains custom UDFs and hence requires a JAR file with
 the classes of the code attached. The constructor of the remote executor takes

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/cluster_setup.md
----------------------------------------------------------------------
diff --git a/docs/cluster_setup.md b/docs/cluster_setup.md
index 7ebbfd2..f0a838b 100644
--- a/docs/cluster_setup.md
+++ b/docs/cluster_setup.md
@@ -2,6 +2,9 @@
 title:  "Cluster Setup"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 This documentation is intended to provide instructions on how to run
 Flink in a fully distributed fashion on a static (but possibly
 heterogeneous) cluster.
@@ -10,9 +13,9 @@ This involves two steps. First, installing and configuring Flink and
 second installing and configuring the [Hadoop Distributed
 Filesystem](http://hadoop.apache.org/) (HDFS).
 
-# Preparing the Cluster
+## Preparing the Cluster
 
-## Software Requirements
+### Software Requirements
 
 Flink runs on all *UNIX-like environments*, e.g. **Linux**, **Mac OS X**,
 and **Cygwin** (for Windows) and expects the cluster to consist of **one master
@@ -29,40 +32,40 @@ install/upgrade it.
 For example, on Ubuntu Linux, type in the following commands to install Java and
 ssh:
 
-```
+~~~bash
 sudo apt-get install ssh 
 sudo apt-get install openjdk-7-jre
-```
+~~~
 
 You can check the correct installation of Java by issuing the following command:
 
-```
+~~~bash
 java -version
-```
+~~~
 
 The command should output something comparable to the following on every node of
 your cluster (depending on your Java version, there may be small differences):
 
-```
+~~~bash
 java version "1.6.0_22"
 Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
 Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)
-```
+~~~
 
 To make sure the ssh daemon is running properly, you can use the command
 
-```
+~~~bash
 ps aux | grep sshd
-```
+~~~
 
 Something comparable to the following line should appear in the output
 of the command on every host of your cluster:
 
-```
+~~~bash
 root       894  0.0  0.0  49260   320 ?        Ss   Jan09   0:13 /usr/sbin/sshd
-```
+~~~
 
-## Configuring Remote Access with ssh
+### Configuring Remote Access with ssh
 
 In order to start/stop the remote processes, the master node requires access via
 ssh to the worker nodes. It is most convenient to use ssh's public key
@@ -79,18 +82,18 @@ public/private key pair into the *.ssh* directory inside the home directory of
 the user *flink*. See the ssh-keygen man page for more details. Note that
 the private key is not protected by a passphrase.
 
-```
+~~~bash
 ssh-keygen -b 2048 -P '' -f ~/.ssh/id_rsa
-```
+~~~
 
 Next, copy/append the content of the file *.ssh/id_rsa.pub* to your
 authorized_keys file. The content of the authorized_keys file defines which
 public keys are considered trustworthy during the public key authentication
 process. On most systems the appropriate command is
 
-```
+~~~bash
 cat .ssh/id_rsa.pub >> .ssh/authorized_keys
-```
+~~~
 
 On some Linux systems, the authorized keys file may also be expected by the ssh
 daemon under *.ssh/authorized_keys2*. In either case, you should make sure the
@@ -100,15 +103,15 @@ node of cluster.
 Finally, the authorized keys file must be copied to every worker node of your
 cluster. You can do this by repeatedly typing in
 
-```
+~~~bash
 scp .ssh/authorized_keys <worker>:~/.ssh/
-```
+~~~
 
 and replacing *\<worker\>* with the host name of the respective worker node.
 After having finished the copy process, you should be able to log on to each
 worker node from your master node via ssh without a password.
 
-## Setting JAVA_HOME on each Node
+### Setting JAVA_HOME on each Node
 
 Flink requires the `JAVA_HOME` environment variable to be set on the
 master and all worker nodes and point to the directory of your Java
@@ -121,20 +124,20 @@ Alternatively, add the following line to your shell profile. If you use the
 *bash* shell (probably the most common shell), the shell profile is located in
 *\~/.bashrc*:
 
-```
+~~~bash
 export JAVA_HOME=/path/to/java_home/
-```
+~~~
 
 If your ssh daemon supports user environments, you can also add `JAVA_HOME` to
 *.\~/.ssh/environment*. As super user *root* you can enable ssh user
 environments with the following commands:
 
-```
+~~~bash
 echo "PermitUserEnvironment yes" >> /etc/ssh/sshd_config
 /etc/init.d/ssh restart
-```
+~~~
 
-# Hadoop Distributed Filesystem (HDFS) Setup
+## Hadoop Distributed Filesystem (HDFS) Setup
 
 The Flink system currently uses the Hadoop Distributed Filesystem (HDFS)
 to read and write data in a distributed fashion.
@@ -146,7 +149,7 @@ many installation guides available online for more detailed instructions.
 **Note that the following instructions are based on Hadoop 1.2 and might differ
 **for Hadoop 2.
 
-## Downloading, Installing, and Configuring HDFS
+### Downloading, Installing, and Configuring HDFS
 
 Similar to the Flink system HDFS runs in a distributed fashion. HDFS
 consists of a **NameNode** which manages the distributed file system's meta
@@ -162,15 +165,17 @@ Next, extract the Hadoop archive.
 After having extracted the Hadoop archive, change into the Hadoop directory and
 edit the Hadoop environment configuration file:
 
-```
+~~~bash
 cd hadoop-*
 vi conf/hadoop-env.sh
-```
+~~~
 
 Uncomment and modify the following line in the file according to the path of
 your Java installation.
 
-``` export JAVA_HOME=/path/to/java_home/ ```
+~~~
+export JAVA_HOME=/path/to/java_home/
+~~~
 
 Save the changes and open the HDFS configuration file *conf/hdfs-site.xml*. HDFS
 offers multiple configuration parameters which affect the behavior of the
@@ -179,7 +184,7 @@ configuration which is required to make HDFS work. More information on how to
 configure HDFS can be found in the [HDFS User
 Guide](http://hadoop.apache.org/docs/r1.2.1/hdfs_user_guide.html) guide.
 
-```xml
+~~~xml
 <configuration>
   <property>
     <name>fs.default.name</name>
@@ -190,7 +195,7 @@ Guide](http://hadoop.apache.org/docs/r1.2.1/hdfs_user_guide.html) guide.
     <value>DATAPATH</value>
   </property>
 </configuration>
-```
+~~~
 
 Replace *MASTER* with the IP/host name of your master node which runs the
 *NameNode*. *DATAPATH* must be replaced with path to the directory in which the
@@ -202,23 +207,23 @@ After having saved the HDFS configuration file, open the file *conf/slaves* and
 enter the IP/host name of those worker nodes which shall act as *DataNode*s.
 Each entry must be separated by a line break.
 
-```
+~~~
 <worker 1>
 <worker 2>
 .
 .
 .
 <worker n>
-```
+~~~
 
 Initialize the HDFS by typing in the following command. Note that the
 command will **delete all data** which has been previously stored in the
 HDFS. However, since we have just installed a fresh HDFS, it should be
 safe to answer the confirmation with *yes*.
 
-```
+~~~bash
 bin/hadoop namenode -format
-```
+~~~
 
 Finally, we need to make sure that the Hadoop directory is available to
 all worker nodes which are intended to act as DataNodes and that all nodes
@@ -227,15 +232,15 @@ directory (e.g. an NFS share) for that. Alternatively, one can copy the
 directory to all nodes (with the disadvantage that all configuration and
 code updates need to be synced to all nodes).
 
-## Starting HDFS
+### Starting HDFS
 
 To start the HDFS log on to the master and type in the following
 commands
 
-```
+~~~bash
 cd hadoop-*
 binn/start-dfs.sh
-```
+~~~
 
 If your HDFS setup is correct, you should be able to open the HDFS
 status website at *http://MASTER:50070*. In a matter of a seconds,
@@ -244,7 +249,7 @@ like to point you to the [Hadoop Quick
 Start](http://wiki.apache.org/hadoop/QuickStart)
 guide.
 
-# Flink Setup
+## Flink Setup
 
 Go to the [downloads page](downloads/) and get the ready to run
 package. Make sure to pick the Flink package **matching your Hadoop
@@ -253,12 +258,12 @@ version**.
 After downloading the latest release, copy the archive to your master node and
 extract it:
 
-```
+~~~bash
 tar xzf flink-*.tgz
 cd flink-*
-```
+~~~
 
-## Configuring the Cluster
+### Configuring the Cluster
 
 After having extracted the system files, you need to configure Flink for
 the cluster by editing *conf/flink-conf.yaml*.
@@ -279,20 +284,20 @@ will later run a TaskManager.
 
 Each entry must be separated by a new line, as in the following example:
 
-```
+~~~
 192.168.0.100
 192.168.0.101
 .
 .
 .
 192.168.0.150
-```
+~~~
 
 The Flink directory must be available on every worker under the same
 path. Similarly as for HDFS, you can use a shared NSF directory, or copy the
 entire Flink directory to every worker node.
 
-## Configuring the Network Buffers
+### Configuring the Network Buffers
 
 Network buffers are a critical resource for the communication layers. They are
 used to buffer records before transmission over a network, and to buffer
@@ -326,7 +331,7 @@ parameters:
 - `taskmanager.network.numberOfBuffers`, and
 - `taskmanager.network.bufferSizeInBytes`.
 
-## Configuring Temporary I/O Directories
+### Configuring Temporary I/O Directories
 
 Although Flink aims to process as much data in main memory as possible,
 it is not uncommon that  more data needs to be processed than memory is
@@ -348,7 +353,7 @@ system, such as */tmp* in Linux systems.
 Please see the [configuration page](config.html) for details and additional
 configuration options.
 
-## Starting Flink
+### Starting Flink
 
 The following script starts a JobManager on the local node and connects via
 SSH to all worker nodes listed in the *slaves* file to start the
@@ -358,6 +363,6 @@ at the configured RPC port.
 
 Assuming that you are on the master node and inside the Flink directory:
 
-```
+~~~bash
 bin/start-cluster.sh
-```
+~~~

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
index 47a096f..ec078d8 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -2,7 +2,10 @@
 title:  "Configuration"
 ---
 
-# Overview
+* This will be replaced by the TOC
+{:toc}
+
+## Overview
 
 The default configuration parameters allow Flink to run out-of-the-box
 in single node setups.
@@ -19,7 +22,7 @@ The system and run scripts parse the config at startup time. Changes to the conf
 file require restarting the Flink JobManager and TaskManagers.
 
 
-# Common Options
+## Common Options
 
 - `env.java.home`: The path to the Java installation to use (DEFAULT: system's
 default Java installation, if found). Needs to be specified if the startup
@@ -71,7 +74,7 @@ and replication factors. Flink will look for the "core-site.xml" and
 "hdfs-site.xml" files in teh specified directory.
 
 
-# Advanced Options
+## Advanced Options
 
 - `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
 directories separated by the systems directory delimiter (for example ':'
@@ -116,9 +119,9 @@ free for objects created by user-defined functions. (DEFAULT: 0.7)
 This parameter is only evaluated, if `taskmanager.memory.size` is not set.
 
 
-# Full Reference
+## Full Reference
 
-## HDFS
+### HDFS
 
 These parameters configure the default HDFS used by Flink. Setups that do not
 specify a HDFS configuration have to specify the full path to 
@@ -133,7 +136,7 @@ directory (DEFAULT: null).
 - `fs.hdfs.hdfssite`: The absolute path of Hadoop's own configuration file
 "hdfs-site.xml" (DEFAULT: null).
 
-## JobManager &amp; TaskManager
+### JobManager &amp; TaskManager
 
 The following parameters configure Flink's JobManager and TaskManagers.
 
@@ -196,7 +199,7 @@ but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128)
 - `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling
 when this fraction of its memory budget is full (DEFAULT: 0.8).
 
-## JobManager Web Frontend
+### JobManager Web Frontend
 
 - `jobmanager.web.port`: Port of the JobManager's web interface that displays
 status of running jobs and execution time breakdowns of finished jobs
@@ -204,7 +207,7 @@ status of running jobs and execution time breakdowns of finished jobs
 - `jobmanager.web.history`: The number of latest jobs that the JobManager's web
 front-end in its history (DEFAULT: 5).
 
-## Webclient
+### Webclient
 
 These parameters configure the web interface that can be used to submit jobs and
 review the compiler's execution plans.
@@ -219,7 +222,7 @@ uploaded programs (DEFAULT: ${webclient.tempdir}/webclient-jobs/).
 temporary JSON files describing the execution plans
 (DEFAULT: ${webclient.tempdir}/webclient-plans/).
 
-## File Systems
+### File Systems
 
 The parameters define the behavior of tasks that create result files.
 
@@ -234,7 +237,7 @@ and place a single result file into it. If the option is set to *false*, the
 writer will directly create the file directly at the output path, without
 creating a containing directory. (DEFAULT: false)
 
-## Compiler/Optimizer
+### Compiler/Optimizer
 
 - `compiler.delimited-informat.max-line-samples`: The maximum number of line
 samples taken by the compiler for delimited inputs. The samples are used to
@@ -250,7 +253,7 @@ sample exceeds this value (possible because of misconfiguration of the parser),
 the sampling aborts. This value can be overridden for a specific input with the
 input format's parameters (DEFAULT: 2097152 (= 2 MiBytes)).
 
-# YARN
+## YARN
 
 Please note that all ports used by Flink in a YARN session are offsetted by the YARN application ID
 to avoid duplicate port allocations when running multiple YARN sessions in parallel. 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/css/codetabs.css
----------------------------------------------------------------------
diff --git a/docs/css/codetabs.css b/docs/css/codetabs.css
new file mode 100644
index 0000000..277a3a6
--- /dev/null
+++ b/docs/css/codetabs.css
@@ -0,0 +1,43 @@
+/**
+ * Make dropdown menus in nav bars show on hover instead of click
+ * using solution at http://stackoverflow.com/questions/8878033/how-
+ * to-make-twitter-bootstrap-menu-dropdown-on-hover-rather-than-click
+ **/
+ul.nav li.dropdown:hover ul.dropdown-menu{
+  display: block;
+}
+
+a.menu:after, .dropdown-toggle:after {
+  content: none;
+}
+
+/** Make the submenus open on hover on the parent menu item */
+ul.nav li.dropdown ul.dropdown-menu li.dropdown-submenu:hover ul.dropdown-menu {
+  display: block;
+}
+
+/** Make the submenus be invisible until the parent menu item is hovered upon */
+ul.nav li.dropdown ul.dropdown-menu li.dropdown-submenu ul.dropdown-menu {
+  display: none;
+}
+
+/**
+ * Made the navigation bar buttons not grey out when clicked.
+ * Essentially making nav bar buttons not react to clicks, only hover events.
+ */
+.navbar .nav li.dropdown.open > .dropdown-toggle {
+  background-color: transparent;
+}
+
+/**
+ * Made the active tab caption blue. Otherwise the active tab is black, and inactive tab is blue.
+ * That looks weird. Changed the colors to active - blue, inactive - black, and
+ * no color change on hover.
+ */
+.nav-tabs > .active > a, .nav-tabs > .active > a:hover {
+  color: #08c;
+}
+
+.nav-tabs > li > a, .nav-tabs > li > a:hover {
+  color: #333;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/dataset_transformations.md
----------------------------------------------------------------------
diff --git a/docs/dataset_transformations.md b/docs/dataset_transformations.md
new file mode 100644
index 0000000..4fa7ff6
--- /dev/null
+++ b/docs/dataset_transformations.md
@@ -0,0 +1,677 @@
+---
+title: "DataSet Transformations"
+---
+
+<section id="top">
+DataSet Transformations
+-----------------------
+
+This document gives a deep-dive into the available transformations on DataSets. For a general introduction to the
+Flink Java API, please refer to the [API guide](java_api_guide.html)
+
+
+### Map
+
+The Map transformation applies a user-defined `MapFunction` on each element of a DataSet.
+It implements a one-to-one mapping, that is, exactly one element must be returned by
+the function.
+
+The following code transforms a `DataSet` of Integer pairs into a `DataSet` of Integers:
+
+```java
+// MapFunction that adds two integer values
+public class IntAdder implements MapFunction<Tuple2<Integer, Integer>, Integer> {
+  @Override
+  public Integer map(Tuple2<Integer, Integer> in) {
+    return in.f0 + in.f1;
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, Integer>> intPairs = // [...]
+DataSet<Integer> intSums = intPairs.map(new IntAdder());
+```
+
+### FlatMap
+
+The FlatMap transformation applies a user-defined `FlatMapFunction` on each element of a `DataSet`.
+This variant of a map function can return arbitrary many result elements (including none) for each input element.
+
+The following code transforms a `DataSet` of text lines into a `DataSet` of words:
+
+```java
+// FlatMapFunction that tokenizes a String by whitespace characters and emits all String tokens.
+public class Tokenizer implements FlatMapFunction<String, String> {
+  @Override
+  public void flatMap(String value, Collector<String> out) {
+    for (String token : value.split("\\W")) {
+      out.collect(token);
+    }
+  }
+}
+
+// [...]
+DataSet<String> textLines = // [...]
+DataSet<String> words = textLines.flatMap(new Tokenizer());
+```
+
+### MapPartition
+
+The MapPartition function transforms a parallel partition in a single function call. The function get the partition as an `Iterable` stream and
+can produce an arbitrary number of result values. The number of elements in each partition depends on the degree-of-parallelism
+and previous operations.
+
+The following code transforms a `DataSet` of text lines into a `DataSet` of counts per partition:
+
+```java
+public class PartitionCounter implements MapPartitionFunction<String, Long> {
+
+  public void mapPartition(Iterable<String> values, Collector<Long> out) {
+    long c = 0;
+    for (String s : values) {
+      c++;
+    }
+    out.collect(c);
+  }
+}
+
+// [...]
+DataSet<String> textLines = // [...]
+DataSet<Long> counts = textLines.mapPartition(new PartitionCounter());
+```
+
+### Filter
+
+The Filter transformation applies a user-defined `FilterFunction` on each element of a `DataSet` and retains only those elements for which the function returns `true`.
+
+The following code removes all Integers smaller than zero from a `DataSet`:
+
+```java
+// FilterFunction that filters out all Integers smaller than zero.
+public class NaturalNumberFilter implements FilterFunction<Integer> {
+  @Override
+  public boolean filter(Integer number) {
+    return number >= 0;
+  }
+}
+
+// [...]
+DataSet<Integer> intNumbers = // [...]
+DataSet<Integer> naturalNumbers = intNumbers.filter(new NaturalNumberFilter());
+```
+
+### Project (Tuple DataSets only)
+
+The Project transformation removes or moves `Tuple` fields of a `Tuple` `DataSet`.
+The `project(int...)` method selects `Tuple` fields that should be retained by their index and defines their order in the output `Tuple`.
+The `types(Class<?> ...)`method must give the types of the output `Tuple` fields.
+
+Projections do not require the definition of a user function.
+
+The following code shows different ways to apply a Project transformation on a `DataSet`:
+
+```java
+DataSet<Tuple3<Integer, Double, String>> in = // [...]
+// converts Tuple3<Integer, Double, String> into Tuple2<String, Integer>
+DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
+```
+
+### Transformations on grouped DataSet
+
+The reduce operations can operate on grouped data sets. Specifying the key to
+be used for grouping can be done in two ways:
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+Please look at the reduce examples to see how the grouping keys are specified.
+
+### Reduce on grouped DataSet
+
+A Reduce transformation that is applied on a grouped `DataSet` reduces each group to a single element using a user-defined `ReduceFunction`.
+For each group of input elements, a `ReduceFunction` successively combines pairs of elements into one element until only a single element for each group remains.
+
+#### Reduce on DataSet grouped by KeySelector Function
+
+A `KeySelector` function extracts a key value from each element of a `DataSet`. The extracted key value is used to group the `DataSet`.
+The following code shows how to group a POJO `DataSet` using a `KeySelector` function and to reduce it with a `ReduceFunction`.
+
+```java
+// some ordinary POJO
+public class WC {
+  public String word;
+  public int count;
+  // [...]
+}
+
+// ReduceFunction that sums Integer attributes of a POJO
+public class WordCounter implements ReduceFunction<WC> {
+  @Override
+  public WC reduce(WC in1, WC in2) {
+    return new WC(in1.word, in1.count + in2.count);
+  }
+}
+
+// [...]
+DataSet<WC> words = // [...]
+DataSet<WC> wordCounts = words
+                         // DataSet grouping with inline-defined KeySelector function
+                         .groupBy(
+                           new KeySelector<WC, String>() {
+                             public String getKey(WC wc) { return wc.word; }
+                           })
+                         // apply ReduceFunction on grouped DataSet
+                         .reduce(new WordCounter());
+```
+
+#### Reduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+
+Field position keys specify one or more fields of a `Tuple` `DataSet` that are used as grouping keys.
+The following code shows how to use field position keys and apply a `ReduceFunction`.
+
+```java
+DataSet<Tuple3<String, Integer, Double>> tuples = // [...]
+DataSet<Tuple3<String, Integer, Double>> reducedTuples =
+                                         tuples
+                                         // group DataSet on first and second field of Tuple
+                                         .groupBy(0,1)
+                                         // apply ReduceFunction on grouped DataSet
+                                         .reduce(new MyTupleReducer());
+```
+
+### GroupReduce on grouped DataSet
+
+A GroupReduce transformation that is applied on a grouped `DataSet` calls a user-defined `GroupReduceFunction` for each group. The difference
+between this and `Reduce` is that the user defined function gets the whole group at once.
+The function is invoked with an Iterable over all elements of a group and can return an arbitrary number of result elements using the collector.
+
+#### GroupReduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+
+The following code shows how duplicate strings can be removed from a `DataSet` grouped by Integer.
+
+```java
+public class DistinctReduce
+         implements GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
+
+  @Override
+  public void reduce(Iterable<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
+
+    Set<String> uniqStrings = new HashSet<String>();
+    Integer key = null;
+  
+    // add all strings of the group to the set
+    for (Tuple2<Integer, String> t : in) {
+      key = t.f0;
+      uniqStrings.add(t.f1);
+    }
+
+    // emit all unique strings.
+    for (String s : uniqStrings) {
+      out.collect(new Tuple2<Integer, String>(key, s));
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, String>> input = // [...]
+DataSet<Tuple2<Integer, String>> output = input
+                           .groupBy(0)            // group DataSet by the first tuple field
+                           .reduceGroup(new DistinctReduce());  // apply GroupReduceFunction
+```
+
+#### GroupReduce on DataSet grouped by KeySelector Function
+
+Works analogous to `KeySelector` functions in Reduce transformations.
+
+#### GroupReduce on sorted groups (Tuple DataSets only)
+
+A `GroupReduceFunction` accesses the elements of a group using an Iterable. Optionally, the Iterable can hand out the elements of a group in a specified order. In many cases this can help to reduce the complexity of a user-defined `GroupReduceFunction` and improve its efficiency.
+Right now, this feature is only available for DataSets of Tuples.
+
+The following code shows another example how to remove duplicate Strings in a `DataSet` grouped by an Integer and sorted by String.
+
+```java
+// GroupReduceFunction that removes consecutive identical elements
+public class DistinctReduce
+         implements GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
+
+  @Override
+  public void reduce(Iterable<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
+    Integer key = null;
+    String comp = null;
+
+    for (Tuple2<Integer, String> t : in) {
+      key = t.f0;
+      String next = t.f1;
+
+      // check if strings are different
+      if (com == null || !next.equals(comp)) {
+        out.collect(new Tuple2<Integer, String>(key, next));
+        comp = next;
+      }
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, String>> input = // [...]
+DataSet<Double> output = input
+                         .groupBy(0)                         // group DataSet by first field
+                         .sortGroup(1, Order.ASCENDING)      // sort groups on second tuple field
+                         .reduceGroup(new DistinctReduce());
+```
+
+**Note:** A GroupSort often comes for free if the grouping is established using a sort-based execution strategy of an operator before the reduce operation.
+
+#### Combinable GroupReduceFunctions
+
+In contrast to a `ReduceFunction`, a `GroupReduceFunction` is not
+necessarily combinable. In order to make a `GroupReduceFunction`
+combinable, you need to use the `RichGroupReduceFunction` variant,
+implement (override) the `combine()` method, and annotate the
+`GroupReduceFunction` with the `@Combinable` annotation as shown here:
+
+```java
+// Combinable GroupReduceFunction that computes two sums.
+// Note that we use the RichGroupReduceFunction because it defines the combine method
+@Combinable
+public class MyCombinableGroupReducer
+         extends RichGroupReduceFunction<Tuple3<String, Integer, Double>,
+                                     Tuple3<String, Integer, Double>> {
+  @Override
+  public void reduce(Iterable<Tuple3<String, Integer, Double>> in,
+                     Collector<Tuple3<String, Integer, Double>> out) {
+
+    String key = null
+    int intSum = 0;
+    double doubleSum = 0.0;
+
+    for (Tuple3<String, Integer, Double> curr : in) {
+      key = curr.f0;
+      intSum += curr.f1;
+      doubleSum += curr.f2;
+    }
+    // emit a tuple with both sums
+    out.collect(new Tuple3<String, Integer, Double>(key, intSum, doubleSum));
+  }
+
+  @Override
+  public void combine(Iterable<Tuple3<String, Integer, Double>> in,
+                      Collector<Tuple3<String, Integer, Double>> out)) {
+    // in some cases combine() calls can simply be forwarded to reduce().
+    this.reduce(in, out);
+  }
+}
+```
+
+### Aggregate on grouped Tuple DataSet
+
+There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+
+- Sum,
+- Min, and
+- Max.
+
+The Aggregate transformation can only be applied on a `Tuple` `DataSet` and supports only field positions keys for grouping.
+
+The following code shows how to apply an Aggregation transformation on a `DataSet` grouped by field position keys:
+
+```java
+DataSet<Tuple3<Integer, String, Double>> input = // [...]
+DataSet<Tuple3<Integer, String, Double>> output = input
+                                   .groupBy(1)        // group DataSet on second field
+                                   .aggregate(SUM, 0) // compute sum of the first field
+                                   .and(MIN, 2);      // compute minimum of the third field
+```
+
+To apply multiple aggregations on a DataSet it is necessary to use the `.and()` function after the first aggregate, that means `.aggregate(SUM, 0).and(MIN, 2)` produces the sum of field 0 and the minimum of field 2 of the original DataSet. 
+In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggregation on an aggregation. In the given example it would produce the minimum of field 2 after calculating the sum of field 0 grouped by field 1.
+
+**Note:** The set of aggregation functions will be extended in the future.
+
+### Reduce on full DataSet
+
+The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a `DataSet`.
+The `ReduceFunction` subsequently combines pairs of elements into one element until only a single element remains.
+
+The following code shows how to sum all elements of an Integer `DataSet`:
+
+```java
+// ReduceFunction that sums Integers
+public class IntSummer implements ReduceFunction<Integer> {
+  @Override
+  public Integer reduce(Integer num1, Integer num2) {
+    return num1 + num2;
+  }
+}
+
+// [...]
+DataSet<Integer> intNumbers = // [...]
+DataSet<Integer> sum = intNumbers.reduce(new IntSummer());
+```
+
+Reducing a full `DataSet` using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
+
+### GroupReduce on full DataSet
+
+The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a `DataSet`.
+A `GroupReduceFunction` can iterate over all elements of `DataSet` and return an arbitrary number of result elements.
+
+The following example shows how to apply a GroupReduce transformation on a full `DataSet`:
+
+```java
+DataSet<Integer> input = // [...]
+// apply a (preferably combinable) GroupReduceFunction to a DataSet
+DataSet<Double> output = input.reduceGroup(new MyGroupReducer());
+```
+
+**Note:** A GroupReduce transformation on a full `DataSet` cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
+
+### Aggregate on full Tuple DataSet
+
+There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+
+- Sum,
+- Min, and
+- Max.
+
+The Aggregate transformation can only be applied on a `Tuple` `DataSet`.
+
+The following code shows how to apply an Aggregation transformation on a full `DataSet`:
+
+```java
+DataSet<Tuple2<Integer, Double>> input = // [...]
+DataSet<Tuple2<Integer, Double>> output = input
+                                     .aggregate(SUM, 0)    // compute sum of the first field
+                                     .and(MIN, 1);    // compute minimum of the second field
+```
+
+**Note:** Extending the set of supported aggregation functions is on our roadmap.
+
+### Join
+
+The Join transformation joins two `DataSet`s into one `DataSet`. The elements of both `DataSet`s are joined on one or more keys which can be specified using
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+There are a few different ways to perform a Join transformation which are shown in the following.
+
+#### Default Join (Join into Tuple2)
+
+The default Join transformation produces a new `Tuple``DataSet` with two fields. Each tuple holds a joined element of the first input `DataSet` in the first tuple field and a matching element of the second input `DataSet` in the second field.
+
+The following code shows a default Join transformation using field position keys:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Double, Integer>> input2 = // [...]
+// result dataset is typed as Tuple2
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Double, Integer>>>
+            result = input1.join(input2)
+                           .where(0)       // key of the first input
+                           .equalTo(1);    // key of the second input
+```
+
+#### Join with JoinFunction
+
+A Join transformation can also call a user-defined `JoinFunction` to process joining tuples.
+A `JoinFunction` receives one element of the first input `DataSet` and one element of the second input `DataSet` and returns exactly one element.
+
+The following code performs a join of `DataSet` with custom java objects and a `Tuple` `DataSet` using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
+
+```java
+// some POJO
+public class Rating {
+  public String name;
+  public String category;
+  public int points;
+}
+
+// Join function that joins a custom POJO with a Tuple
+public class PointWeighter
+         implements JoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
+
+  @Override
+  public Tuple2<String, Double> join(Rating rating, Tuple2<String, Double> weight) {
+    // multiply the points and rating and construct a new output tuple
+    return new Tuple2<String, Double>(rating.name, rating.points * weight.f1);
+  }
+}
+
+DataSet<Rating> ratings = // [...]
+DataSet<Tuple2<String, Double>> weights = // [...]
+DataSet<Tuple2<String, Double>>
+            weightedRatings =
+            ratings.join(weights)
+
+                   // key of the first input
+                   .where(new KeySelection<Rating, String>() {
+                            public String getKey(Rating r) { return r.category; }
+                          })
+
+                   // key of the second input
+                   .equalTo(new KeySelection<Tuple2<String, Double>, String>() {
+                              public String getKey(Tuple2<String, Double> t) { return t.f0; }
+                            })
+
+                   // applying the JoinFunction on joining pairs
+                   .with(new PointWeighter());
+```
+
+#### Join with FlatJoinFunction
+
+Analogous to Map and FlatMap, a FlatJoin function behaves in the same
+way as a JoinFunction, but instead of returning one element, it can
+return (collect), zero, one, or more elements.
+{% highlight java %}
+public class PointWeighter
+         implements FlatJoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
+  @Override
+  public void join(Rating rating, Tuple2<String, Double> weight,
+	  Collector<Tuple2<String, Double>> out) {
+	if (weight.f1 > 0.1) {
+		out.collect(new Tuple2<String, Double>(rating.name, rating.points * weight.f1));
+	}
+  }
+}
+
+DataSet<Tuple2<String, Double>>
+            weightedRatings =
+            ratings.join(weights) // [...]
+{% endhighlight %}
+
+#### Join with Projection
+
+A Join transformation can construct result tuples using a projection as shown here:
+
+```java
+DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
+DataSet<Tuple2<Integer, Double>> input2 = // [...]
+DataSet<Tuple4<Integer, String, Double, Byte>
+            result =
+            input1.join(input2)
+                  // key definition on first DataSet using a field position key
+                  .where(0)
+                  // key definition of second DataSet using a field position key
+                  .equalTo(0)
+                  // select and reorder fields of matching tuples
+                  .projectFirst(0,2).projectSecond(1).projectFirst(1)
+                  .types(Integer.class, String.class, Double.class, Byte.class);
+```
+
+`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output `Tuple`. The order of indexes defines the order of fields in the output tuple.
+The join projection works also for non-`Tuple` `DataSet`s. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output `Tuple`.
+
+#### Join with DataSet Size Hint
+
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to join as shown here:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Integer, String>> input2 = // [...]
+
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
+            result1 =
+            // hint that the second DataSet is very small
+            input1.joinWithTiny(input2)
+                  .where(0)
+                  .equalTo(0);
+
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
+            result2 =
+            // hint that the second DataSet is very large
+            input1.joinWithHuge(input2)
+                  .where(0)
+                  .equalTo(0);
+```
+
+### Cross
+
+The Cross transformation combines two `DataSet`s into one `DataSet`. It builds all pairwise combinations of the elements of both input `DataSet`s, i.e., it builds a Cartesian product.
+The Cross transformation either calls a user-defined `CrossFunction` on each pair of elements or applies a projection. Both modes are shown in the following.
+
+**Note:** Cross is potentially a *very* compute-intensive operation which can challenge even large compute clusters!
+
+#### Cross with User-Defined Function
+
+A Cross transformation can call a user-defined `CrossFunction`. A `CrossFunction` receives one element of the first input and one element of the second input and returns exactly one result element.
+
+The following code shows how to apply a Cross transformation on two `DataSet`s using a `CrossFunction`:
+
+```java
+public class Coord {
+  public int id;
+  public int x;
+  public int y;
+}
+
+// CrossFunction computes the Euclidean distance between two Coord objects.
+public class EuclideanDistComputer
+         implements CrossFunction<Coord, Coord, Tuple3<Integer, Integer, Double>> {
+
+  @Override
+  public Tuple3<Integer, Integer, Double> cross(Coord c1, Coord c2) {
+    // compute Euclidean distance of coordinates
+    double dist = sqrt(pow(c1.x - c2.x, 2) + pow(c1.y - c2.y, 2));
+    return new Tuple3<Integer, Integer, Double>(c1.id, c2.id, dist);
+  }
+}
+
+DataSet<Coord> coords1 = // [...]
+DataSet<Coord> coords2 = // [...]
+DataSet<Tuple3<Integer, Integer, Double>>
+            distances =
+            coords1.cross(coords2)
+                   // apply CrossFunction
+                   .with(new EuclideanDistComputer());
+```
+
+#### Cross with Projection
+
+A Cross transformation can also construct result tuples using a projection as shown here:
+
+```java
+DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
+DataSet<Tuple2<Integer, Double>> input2 = // [...]
+DataSet<Tuple4<Integer, Byte, Integer, Double>
+            result =
+            input1.cross(input2)
+                  // select and reorder fields of matching tuples
+                  .projectSecond(0).projectFirst(1,0).projectSecond(1)
+                  .types(Integer.class, Byte.class, Integer.class, Double.class);
+```
+
+The field selection in a Cross projection works the same way as in the projection of Join results.
+
+#### Cross with DataSet Size Hint
+
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to cross as shown here:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Integer, String>> input2 = // [...]
+
+DataSet<Tuple4<Integer, String, Integer, String>>
+            udfResult =
+                  // hint that the second DataSet is very small
+            input1.crossWithTiny(input2)
+                  // apply any Cross function (or projection)
+                  .with(new MyCrosser());
+
+DataSet<Tuple3<Integer, Integer, String>>
+            projectResult =
+                  // hint that the second DataSet is very large
+            input1.crossWithHuge(input2)
+                  // apply a projection (or any Cross function)
+                  .projectFirst(0,1).projectSecond(1).types(Integer.class, String.class, String.class)
+```
+
+### CoGroup
+
+The CoGroup transformation jointly processes groups of two `DataSet`s. Both `DataSet`s are grouped on a defined key and groups of both `DataSet`s that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one `DataSet` has a group, the `CoGroupFunction` is called with this group and an empty group.
+A `CoGroupFunction` can separately iterate over the elements of both groups and return an arbitrary number of result elements.
+
+Similar to Reduce, GroupReduce, and Join, keys can be defined using
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+#### CoGroup on DataSets grouped by Field Position Keys (Tuple DataSets only)
+
+```java
+// Some CoGroupFunction definition
+class MyCoGrouper
+         implements CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Double>, Double> {
+
+  @Override
+  public void coGroup(Iterable<Tuple2<String, Integer>> iVals,
+                      Iterable<Tuple2<String, Double>> dVals,
+                      Collector<Double> out) {
+
+    Set<Integer> ints = new HashSet<Integer>();
+
+    // add all Integer values in group to set
+    for (Tuple2<String, Integer>> val : iVale) {
+      ints.add(val.f1);
+    }
+
+    // multiply each Double value with each unique Integer values of group
+    for (Tuple2<String, Double> val : dVals) {
+      for (Integer i : ints) {
+        out.collect(val.f1 * i);
+      }
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<String, Integer>> iVals = // [...]
+DataSet<Tuple2<String, Double>> dVals = // [...]
+DataSet<Double> output = iVals.coGroup(dVals)
+                         // group first DataSet on first tuple field
+                         .where(0)
+                         // group second DataSet on first tuple field
+                         .equalTo(0)
+                         // apply CoGroup function on each pair of groups
+                         .with(new MyCoGrouper());
+```
+
+#### CoGroup on DataSets grouped by Key Selector Function
+
+Works analogous to key selector functions in Join transformations.
+
+### Union
+
+Produces the union of two `DataSet`s, which have to be of the same type. A union of more than two `DataSet`s can be implemented with multiple union calls, as shown here:
+
+```java
+DataSet<Tuple2<String, Integer>> vals1 = // [...]
+DataSet<Tuple2<String, Integer>> vals2 = // [...]
+DataSet<Tuple2<String, Integer>> vals3 = // [...]
+DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2)
+                    .union(vals3);
+```
+
+
+[Back to top](#top)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/example_connectors.md
----------------------------------------------------------------------
diff --git a/docs/example_connectors.md b/docs/example_connectors.md
index 706bf99..29d8f1d 100644
--- a/docs/example_connectors.md
+++ b/docs/example_connectors.md
@@ -18,43 +18,45 @@ This example is using the `HadoopInputFormat` wrapper to use an existing Hadoop
 1. Download and compile the `azure-tables-hadoop` project. The input format developed by the project is not yet available in Maven Central, therefore, we have to build the project ourselves.
 Execute the following commands:
 
-    ```bash
-    git clone https://github.com/mooso/azure-tables-hadoop.git
-    cd azure-tables-hadoop
-    mvn clean install
-    ```
+   ~~~bash
+   git clone https://github.com/mooso/azure-tables-hadoop.git
+   cd azure-tables-hadoop
+   mvn clean install
+   ~~~
 
 2. Setup a new Flink project using the quickstarts:
 
-    ```bash
-    curl https://raw.githubusercontent.com/apache/incubator-flink/master/flink-quickstart/quickstart.sh | bash
-    ```
+   ~~~bash
+   curl https://raw.githubusercontent.com/apache/incubator-flink/master/flink-quickstart/quickstart.sh | bash
+   ~~~
 
 3. Set the the version of Flink to `{{site.FLINK_VERSION_HADOOP_2_STABLE}}` in the `pom.xml` file. The quickstart.sh script sets the version to the `hadoop1` version of Flink. Since the `microsoft-hadoop-azure` has been written for Hadoop 2.2 (mapreduce-API) version, we need to use the appropriate Flink version. 
 
     Replace all occurences of `<version>{{site.FLINK_VERSION_STABLE}}</version>` with `<version>{{site.FLINK_VERSION_HADOOP_2_STABLE}}</version>`.
 4. Add the following dependencies (in the `<dependencies>` section) to your `pom.xml` file:
 
-    ```xml
-    <dependency>
-    	<groupId>org.apache.flink</groupId>
-    	<artifactId>flink-hadoop-compatibility</artifactId>
-    	<version>{{site.FLINK_VERSION_HADOOP_2_STABLE}}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.microsoft.hadoop</groupId>
-      <artifactId>microsoft-hadoop-azure</artifactId>
-      <version>0.0.4</version>
-    </dependency>
-    ```
-    - `flink-hadoop-compatibility` is a Flink package that provides the Hadoop input format wrappers.
-    - `microsoft-hadoop-azure` is adding the project we've build before to our project.
+   ~~~xml
+   <dependency>
+       <groupId>org.apache.flink</groupId>
+       <artifactId>flink-hadoop-compatibility</artifactId>
+       <version>{{site.FLINK_VERSION_HADOOP_2_STABLE}}</version>
+   </dependency>
+   <dependency>
+     <groupId>com.microsoft.hadoop</groupId>
+     <artifactId>microsoft-hadoop-azure</artifactId>
+     <version>0.0.4</version>
+   </dependency>
+   ~~~
+
+   `flink-hadoop-compatibility` is a Flink package that provides the Hadoop input format wrappers.
+   `microsoft-hadoop-azure` is adding the project we've build before to our project.
 
 The project is now prepared for starting to code. We recommend to import the project into an IDE, such as Eclipse or IntelliJ. (Import as a Maven project!).
 Browse to the code of the `Job.java` file. Its an empty skeleton for a Flink job.
 
 Paste the following code into it:
-```java
+
+~~~java
 import java.util.Map;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
@@ -107,7 +109,8 @@ public class AzureTableExample {
     env.execute("Azure Example");
   }
 }
-```
+~~~
+
 The example shows how to access an Azure table and turn data into Flink's `DataSet` (more specifically, the type of the set is `DataSet<Tuple2<Text, WritableEntity>>`). With the `DataSet`, you can apply all known transformations to the DataSet.
 
 ## Access MongoDB

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
index 60fbc5e..1008499 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -2,9 +2,12 @@
 title: "Frequently Asked Questions (FAQ)"
 ---
 
-# General
+* This will be replaced by the TOC
+{:toc}
 
-## Is Flink a Hadoop Project?
+## General
+
+### Is Flink a Hadoop Project?
 
 Flink is a data processing system and an alternative to Hadoop's
 MapReduce component. It comes with its own runtime, rather than building on top
@@ -15,7 +18,7 @@ manager (YARN) to provision cluster resources. Since most Flink users are
 using Hadoop HDFS to store their data, we ship already the required libraries to
 access HDFS.
 
-## Do I have to install Apache Hadoop to use Flink?
+### Do I have to install Apache Hadoop to use Flink?
 
 No. Flink can run without a Hadoop installation. However, a very common
 setup is to use Flink to analyze data stored in the Hadoop Distributed
@@ -28,9 +31,9 @@ YARN](http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-site/YARN.htm
 is Hadoop's cluster resource manager that allows to use
 different execution engines next to each other on a cluster.
 
-# Usage
+## Usage
 
-## How do I assess the progress of a Flink program?
+### How do I assess the progress of a Flink program?
 
 There are a multiple of ways to track the progress of a Flink program:
 
@@ -41,7 +44,7 @@ to observe program execution. In runs on port 8081 by default (configured in
 changes of all operators as the program progresses through the operations.
 - All status changes are also logged to the JobManager's log file.
 
-## How can I figure out why a program failed?
+### How can I figure out why a program failed?
 
 - Thw JobManager web frontend (by default on port 8081) displays the exceptions
 of failed tasks.
@@ -54,7 +57,7 @@ of the master and the worker where the exception occurred
 (`log/flink-<user>-jobmanager-<host>.log` and
 `log/flink-<user>-taskmanager-<host>.log`).
 
-## How do I debug Flink programs?
+### How do I debug Flink programs?
 
 - When you start a program locally with the [LocalExecutor](local_execution.html),
 you can place breakpoints in your functions and debug them like normal
@@ -64,9 +67,9 @@ tracking the behavior of the parallel execution. They allow you to gather
 information inside the program's operations and show them after the program
 execution.
 
-# Errors
+## Errors
 
-## I get an error message saying that not enough buffers are available. How do I fix this?
+### I get an error message saying that not enough buffers are available. How do I fix this?
 
 If you run Flink in a massively parallel setting (100+ parallel threads),
 you need to adapt the number of network buffers via the config parameter
@@ -75,7 +78,7 @@ As a rule-of-thumb, the number of buffers should be at least
 `4 * numberOfNodes * numberOfTasksPerNode^2`. See
 [Configuration Reference](config.html) for details.
 
-## My job fails early with a java.io.EOFException. What could be the cause?
+### My job fails early with a java.io.EOFException. What could be the cause?
 
 Note: In version _0.4_, the delta iterations limit the solution set to
 records with fixed-length data types. We will  in the next version.
@@ -85,7 +88,7 @@ wrong HDFS version. Because different HDFS versions are often not compatible
 with each other, the connection between the filesystem master and the client
 breaks.
 
-```bash
+~~~bash
 Call to <host:port> failed on local exception: java.io.EOFException
     at org.apache.hadoop.ipc.Client.wrapException(Client.java:775)
     at org.apache.hadoop.ipc.Client.call(Client.java:743)
@@ -97,13 +100,13 @@ Call to <host:port> failed on local exception: java.io.EOFException
     at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:170)
     at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:82)
     at org.apache.flinkruntime.fs.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:276
-```
+~~~
 
 Please refer to the [download page]({{site.baseurl}}/downloads.html#maven) and
 the {% gh_link README.md master "build instructions" %}
 for details on how to set up Flink for different Hadoop and HDFS versions.
 
-## My program does not compute the correct result. Why are my custom key types
+### My program does not compute the correct result. Why are my custom key types
 are not grouped/joined correctly?
 
 Keys must correctly implement the methods `java.lang.Object#hashCode()`,
@@ -111,14 +114,14 @@ Keys must correctly implement the methods `java.lang.Object#hashCode()`,
 These methods are always backed with default implementations which are usually
 inadequate. Therefore, all keys must override `hashCode()` and `equals(Object o)`.
 
-## I get a java.lang.InstantiationException for my data type, what is wrong?
+### I get a java.lang.InstantiationException for my data type, what is wrong?
 
 All data type classes must be public and have a public nullary constructor
 (constructor with no arguments). Further more, the classes must not be abstract
 or interfaces. If the classes are internal classes, they must be public and
 static.
 
-## I can't stop Flink with the provided stop-scripts. What can I do?
+### I can't stop Flink with the provided stop-scripts. What can I do?
 
 Stopping the processes sometimes takes a few seconds, because the shutdown may
 do some cleanup work.
@@ -136,7 +139,7 @@ affected JobManager or TaskManager process.
 On Windows, the TaskManager shows a table of all processes and allows you to
 destroy a process by right its entry.
 
-## I got an OutOfMemoryException. What can I do?
+### I got an OutOfMemoryException. What can I do?
 
 These exceptions occur usually when the functions in the program consume a lot
 of memory by collection large numbers of objects, for example in lists or maps.
@@ -162,21 +165,21 @@ entries `taskmanager.memory.fraction` or `taskmanager.memory.size`. See the
 [Configuration Reference](config.html) for details. This will leave more memory to JVM heap,
 but may cause data processing tasks to go to disk more often.
 
-## Why do the TaskManager log files become so huge?
+### Why do the TaskManager log files become so huge?
 
 Check the logging behavior of your jobs. Emitting logging per or tuple may be
 helpful to debug jobs in small setups with tiny data sets, it becomes very
 inefficient and disk space consuming if used for large input data.
 
-# YARN Deployment
+## YARN Deployment
 
-## The YARN session runs only for a few seconds
+### The YARN session runs only for a few seconds
 
 The `./bin/yarn-session.sh` script is intended to run while the YARN-session is
 open. In some error cases however, the script immediately stops running. The
 output looks like this:
 
-```
+~~~
 07:34:27,004 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Submitted application application_1395604279745_273123 to ResourceManager at jobtracker-host
 Flink JobManager is now running on worker1:6123
 JobManager Web Interface: http://jobtracker-host:54311/proxy/application_1295604279745_273123/
@@ -185,7 +188,7 @@ JobManager Web Interface: http://jobtracker-host:54311/proxy/application_1295604
 07:34:51,529 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Killing application application_1295604279745_273123
 07:34:51,534 INFO  org.apache.flinkyarn.Client                                   - Deleting files in hdfs://user/marcus/.flink/application_1295604279745_273123
 07:34:51,559 INFO  org.apache.flinkyarn.Client                                   - YARN Client is shutting down
-```
+~~~
 
 The problem here is that the Application Master (AM) is stopping and the YARN client assumes that the application has finished.
 
@@ -206,11 +209,11 @@ YARN configuration is wrong and more memory than physically available is
 configured. Execute `dmesg` on the machine where the AM was running to see if
 this happened. You see messages from Linux' [OOM killer](http://linux-mm.org/OOM_Killer).
 
-## The YARN session crashes with a HDFS permission exception during startup
+### The YARN session crashes with a HDFS permission exception during startup
 
 While starting the YARN session, you are receiving an exception like this:
 
-```
+~~~
 Exception in thread "main" org.apache.hadoop.security.AccessControlException: Permission denied: user=robert, access=WRITE, inode="/user/robert":hdfs:supergroup:drwxr-xr-x
   at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:234)
   at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:214)
@@ -258,7 +261,7 @@ Exception in thread "main" org.apache.hadoop.security.AccessControlException: Pe
   at org.apache.flinkyarn.Utils.setupLocalResource(Utils.java:176)
   at org.apache.flinkyarn.Client.run(Client.java:362)
   at org.apache.flinkyarn.Client.main(Client.java:568)
-```
+~~~
 
 The reason for this error is, that the home directory of the user **in HDFS**
 has the wrong permissions. The user (in this case `robert`) can not create
@@ -267,14 +270,14 @@ directories in his own home directory.
 Flink creates a `.flink/` directory in the users home directory
 where it stores the Flink jar and configuration file.
 
-# Features
+## Features
 
-## What kind of fault-tolerance does Flink provide?
+### What kind of fault-tolerance does Flink provide?
 
 Flink can restart failed jobs. Mid-query fault tolerance will go into the
 open source project in the next versions.
 
-## Are Hadoop-like utilities, such as Counters and the DistributedCache supported?
+### Are Hadoop-like utilities, such as Counters and the DistributedCache supported?
 
 [Flink's Accumulators](java_api_guide.html#accumulators-&-counters) work very similar like
 [Hadoop's counters, but are more powerful.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/internal_add_operator.md
----------------------------------------------------------------------
diff --git a/docs/internal_add_operator.md b/docs/internal_add_operator.md
index f3c9758..c9ee3bc 100644
--- a/docs/internal_add_operator.md
+++ b/docs/internal_add_operator.md
@@ -2,6 +2,9 @@
 title:  "How to add a new Operator"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 Operators in the Java API can be added in multiple different ways: 
 
 1. On the DataSet, as a specialization/combination of existing operators
@@ -16,20 +19,23 @@ new functionality does require a new runtime operator, or it is much more effici
 
 Many operators can be implemented as a specialization of another operator, or by means of a UDF.
 
-The simplest example are the `sum()`, `min()`, and `max()` functions on the {% gh_link /flink-java/src/main/java/org/apache/flink/api/java/DataSet.java "DataSet" %}. These functions simply call other operations
-with some pre-defined parameters:
-```
+The simplest example are the `sum()`, `min()`, and `max()` functions on the
+{% gh_link /flink-java/src/main/java/org/apache/flink/api/java/DataSet.java "DataSet" %}.
+These functions simply call other operations with some pre-defined parameters:
+
+~~~java
 public AggregateOperator<T> sum (int field) {
     return this.aggregate (Aggregations.SUM, field);
 }
 
-```
+~~~
 
 Some operations can be implemented as compositions of multiple other operators. An example is to implement a
 *count()* function through a combination of *map* and *aggregate*. 
 
 A simple way to do this is to define a function on the {% gh_link /flink-java/src/main/java/org/apache/flink/api/java/DataSet.java "DataSet" %} that calls *map(...)* and *reduce(...)* in turn:
-```
+
+~~~java
 public DataSet<Long> count() {
     return this.map(new MapFunction<T, Long>() {
                         public Long map(T value) {
@@ -42,15 +48,16 @@ public DataSet<Long> count() {
                         }
                     });
 }
-```
+~~~
 
 To define a new operator without altering the DataSet class is possible by putting the functions as static members
 into another class. The example of the *count()* operator would look the following way:
-```
+
+~~~java
 public static <T>DataSet<Long> count(DataSet<T> data) {
     return data.map(...).reduce(...);
 }
-```
+~~~
 
 ### More Complex Operators
 
@@ -68,15 +75,16 @@ Because the operation is translated into a GroupReduce operation, it appears as
 
 The DataSet offers a method for custom operators: `DataSet<X> runOperation(CustomUnaryOperation<T, X> operation)`.
 The *CustomUnaryOperation* interface defines operators by means of the two functions:
-``` java
+
+~~~ java
 void setInput(DataSet<IN> inputData);
 	
 DataSet<OUT> createResult();
-```
+~~~
 
 The {% gh_link /flink-addons/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java "VertexCentricIteration" %} operator is implemented that way. Below is an example how to implement the *count()* operator that way.
 
-``` java
+~~~ java
 public class Counter<T> implements CustomUnaryOperation<T, Long> {
 
     private DataSet<T> input;
@@ -87,12 +95,14 @@ public class Counter<T> implements CustomUnaryOperation<T, Long> {
         return input.map(...).reduce(...);
     }
 }
-```
+~~~
+
 The CountOperator can be called in the following way:
-``` java
+
+~~~ java
 DataSet<String> lines = ...;
 DataSet<Long> count = lines.runOperation(new Counter<String>());
-```
+~~~
 
 
 ## Implementing a new Runtime Operator
@@ -114,7 +124,8 @@ Runtime Operators are implemented using the {% gh_link /flink-runtime/src/main/j
 The runtime works with the `MutableObjectIterator`, which describes data streams with the ability to reuse objects, to reduce pressure on the garbage collector.
 
 An implementation of the central `run()` method for the *mapPartition* operator could look the following way:
-``` java
+
+~~~ java
 public void run() throws Exception {
     final MutableObjectIterator<IN> input = this.taskContext.getInput(0);
     final MapPartitionFunction<IN, OUT> function = this.taskContext.getStub();
@@ -126,7 +137,7 @@ public void run() throws Exception {
 
     function.mapPartition(iterator, output);
 }
-```
+~~~
 
 To increase efficiency, it is often beneficial to implement a *chained* version of an operator. Chained
 operators run in the same thread as their preceding operator, and work with nested function calls.
@@ -136,22 +147,23 @@ To learn how to implement a chained operator, take a look at the {% gh_link /fli
 {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java "ChainedMapDriver" %} (chained variant).
 
 
-**Optimizer/Compiler**
+### Optimizer/Compiler
 
 This section does a minimal discussion of the important steps to add an operator. Please see the [Optimizer](internal_optimizer.html) docs for more detail on how the optimizer works.
 To allow the optimizer to include a new operator in its planning, it needs a bit of information about it; in particular, the following information:
 
 - *{% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java "DriverStrategy" %}*: The operation needs to be added to the Enum, to make it available to the optimizer. The parameters to the Enum entry define which class implements the runtime operator, its chained version, whether the operator accumulates records (and needs memory for that), and whether it requires a comparator (works on keys). For our example, we can add the entry
-``` java
+~~~ java
 MAP_PARTITION(MapPartitionDriver.class, null /* or chained variant */, PIPELINED, false)
-```
+~~~
 
 - *Cost function*: The class {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/costs/CostEstimator.java "CostEstimator" %} needs to know how expensive the operation is to the system. The costs here refer to the non-UDF part of the operator. Since the operator does essentially no work (it forwards the record stream to the UDF), the costs are zero. We change the `costOperator(...)` method by adding the *MAP_PARTITION* constant to the switch statement similar to the *MAP* constant such that no cost is accounted for it.
 
 - *OperatorDescriptor*: The operator descriptors define how an operation needs to be treated by the optimizer. They describe how the operation requires the input data to be (e.g., sorted or partitioned) and that way allows the optimizer to optimize the data movement, sorting, grouping in a global fashion. They do that by describing which {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedGlobalProperties.java "RequestedGlobalProperties" %} (partitioning, replication, etc) and which {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedLocalProperties.java "RequestedLocalProperties" %} (sorting, grouping, uniqueness) the operator has, as well as how the operator affects the existing {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/GlobalProperties.java "GlobalProperties" %} and {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/LocalProperties.j
 ava "LocalProperties" %}. In addition, it defines a few utility methods, for example to instantiate an operator candidate.
 Since the *mapPartition()* function is very simple (no requirements on partitioning/grouping), the descriptor is very simple. Other operators have more complex requirements, for example the {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/operators/GroupReduceProperties.java "GroupReduce" %}. Some operators, like *join* have multiple ways in which they can be executed and therefore have multiple descriptors ({% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/operators/HashJoinBuildFirstProperties.java "Hash Join 1" %}, {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/operators/HashJoinBuildSecondProperties.java "Hash Join 2" %}, {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/operators/SortMergeJoinDescriptor.java "SortMerge Join" %}).
-The code sample below explains (with comments) how to create a descriptor for the *MapPartitionOperator*
-``` java
+The code sample below explains (with comments) how to create a descriptor for the *MapPartitionOperator* 
+
+  ~~~ java
     public DriverStrategy getStrategy() {
         return MAP_PARTITION;
     }
@@ -183,12 +195,12 @@ The code sample below explains (with comments) how to create a descriptor for th
     public LocalProperties computeLocalProperties(LocalProperties lProps) {
         return LocalProperties.EMPTY;
     }
-```
+  ~~~
 
 - *OptimizerNode*: The optimizer node is the place where all comes together. It creates the list of *OperatorDescriptors*, implements the result data set size estimation, and assigns a name to the operation. It is a relatively small class and can be more or less copied again from the {% gh_link /flink-compiler/src/main/java/org/apache/flink/compiler/dag/MapNode.java "MapNode" %}.
 
 
-**Common API**
+### Common API
 
 To make the operation available to the higher-level APIs, it needs to be added to the Common API. The simplest way to do this is to add a
 base operator. Create a class `MapPartitionOperatorBase`, after the pattern of the {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java "MapOperatorBase" %}.
@@ -201,15 +213,16 @@ same function. The Common API operator exists only in order for the `flink-java`
 optimizer.
 
 
-**Java API**
+### Java API
 
 Create a Java API operator that is constructed in the same way as the {% gh_link /flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java "MapOperator" %}. The core method is the `translateToDataFlow(...)` method, which creates the Common API operator for the Java API operator.
 
 The final step is to add a function to the `DataSet` class:
-``` java
+
+~~~ java
 public <R> DataSet<R> mapPartition(MapPartitionFunction<T, R> function) {
     return new MapPartitionOperator<T, R>(this, function);
 }
-```
+~~~
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/internal_general_arch.md
----------------------------------------------------------------------
diff --git a/docs/internal_general_arch.md b/docs/internal_general_arch.md
index ce6e810..d005949 100644
--- a/docs/internal_general_arch.md
+++ b/docs/internal_general_arch.md
@@ -2,6 +2,9 @@
 title:  "General Architecture and Process Model"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 ## The Processes
 
 When the Flink system is started, it bring up the *JobManager* and one or more *TaskManagers*. The JobManager

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/internal_logging.md
----------------------------------------------------------------------
diff --git a/docs/internal_logging.md b/docs/internal_logging.md
index 6d95360..29716fe 100644
--- a/docs/internal_logging.md
+++ b/docs/internal_logging.md
@@ -2,6 +2,9 @@
 title: "How to use logging"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 The logging in Flink is implemented using the slf4j logging interface. As underlying logging framework, logback is used.
 
 ## Configuring logback
@@ -12,7 +15,7 @@ The configuration file either has to be specified by setting the environment pro
 The `conf` directory contains a `logback.xml` file which can be modified and is used if Flink is started outside of an IDE and with the provided starting scripts.
 The provided `logback.xml` has the following form:
 
-``` xml
+~~~ xml
 <configuration>
     <appender name="file" class="ch.qos.logback.core.FileAppender">
         <file>${log.file}</file>
@@ -26,36 +29,39 @@ The provided `logback.xml` has the following form:
         <appender-ref ref="file"/>
     </root>
 </configuration>
-```
+~~~
 
 In order to control the logging level of `org.apache.flink.runtime.jobgraph.JobGraph`, for example, one would have to add the following line to the configuration file.
-``` xml
+
+~~~ xml
 <logger name="org.apache.flink.runtime.jobgraph.JobGraph" level="DEBUG"/>
-```
+~~~
 
 For further information on configuring logback see [LOGback's manual](http://logback.qos.ch/manual/configuration.html).
 
 ## Best practices for developers
 
 The loggers using slf4j are created by calling
-``` java
+
+~~~ java
 import org.slf4j.LoggerFactory
 import org.slf4j.Logger
 
 Logger LOG = LoggerFactory.getLogger(Foobar.class)
-```
+~~~
 
 In order to benefit most from slf4j, it is recommended to use its placeholder mechanism.
 Using placeholders allows to avoid unnecessary string constructions in case that the logging level is set so high that the message would not be logged.
 The syntax of placeholders is the following:
-``` java
+
+~~~ java
 LOG.info("This message contains {} placeholders. {}", 2, "Yippie");
-```
+~~~
 
 Placeholders can also be used in conjunction with exceptions which shall be logged.
 
-``` java
+~~~ java
 catch(Exception exception){
 	LOG.error("An {} occurred.", "error", exception);
 }
-```
\ No newline at end of file
+~~~
\ No newline at end of file


[05/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
new file mode 100644
index 0000000..bbb9e73
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.api.scala.codegen
+
+import scala.language.postfixOps
+
+import scala.reflect.macros.Context
+import scala.reflect.classTag
+import scala.reflect.ClassTag
+import scala.Option.option2Iterable
+
+// These are only used internally while analyzing Scala types in TypeAnalyzer and TypeInformationGen
+
+private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C] =>
+  import c.universe._
+
+  abstract sealed class UDTDescriptor {
+    val id: Int
+    val tpe: Type
+    val isPrimitiveProduct: Boolean = false
+    
+    def canBeKey: Boolean
+
+    def mkRoot: UDTDescriptor = this
+
+    def flatten: Seq[UDTDescriptor]
+    def getters: Seq[FieldAccessor] = Seq()
+
+    def select(member: String): Option[UDTDescriptor] =
+      getters find { _.getter.name.toString == member } map { _.desc }
+    
+    def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
+      case Nil => Seq(Some(this))
+      case head :: tail => getters find { _.getter.name.toString == head } match {
+        case None => Seq(None)
+        case Some(d : FieldAccessor) => d.desc.select(tail)
+      }
+    }
+
+    def findById(id: Int): Option[UDTDescriptor] = flatten.find { _.id == id }
+
+    def findByType[T <: UDTDescriptor: ClassTag]: Seq[T] = {
+      val clazz = classTag[T].runtimeClass
+      flatten filter { item => clazz.isAssignableFrom(item.getClass) } map { _.asInstanceOf[T] }
+    }
+
+    def getRecursiveRefs: Seq[UDTDescriptor] =
+      findByType[RecursiveDescriptor].flatMap { rd => findById(rd.refId) }.map { _.mkRoot }.distinct
+  }
+
+  case class GenericClassDescriptor(id: Int, tpe: Type) extends UDTDescriptor {
+    override def flatten = Seq(this)
+
+    def canBeKey = false
+  }
+
+  case class UnsupportedDescriptor(id: Int, tpe: Type, errors: Seq[String]) extends UDTDescriptor {
+    override def flatten = Seq(this)
+    
+    def canBeKey = tpe <:< typeOf[Comparable[_]]
+  }
+
+  case class PrimitiveDescriptor(id: Int, tpe: Type, default: Literal, wrapper: Type)
+      extends UDTDescriptor {
+    override val isPrimitiveProduct = true
+    override def flatten = Seq(this)
+    override def canBeKey = wrapper <:< typeOf[org.apache.flink.types.Key[_]]
+  }
+
+  case class BoxedPrimitiveDescriptor(
+      id: Int, tpe: Type, default: Literal, wrapper: Type, box: Tree => Tree, unbox: Tree => Tree)
+    extends UDTDescriptor {
+
+    override val isPrimitiveProduct = true
+    override def flatten = Seq(this)
+    override def canBeKey = wrapper <:< typeOf[org.apache.flink.types.Key[_]]
+
+    override def hashCode() = (id, tpe, default, wrapper, "BoxedPrimitiveDescriptor").hashCode()
+    override def equals(that: Any) = that match {
+      case BoxedPrimitiveDescriptor(thatId, thatTpe, thatDefault, thatWrapper, _, _) =>
+        (id, tpe, default, wrapper).equals(thatId, thatTpe, thatDefault, thatWrapper)
+      case _ => false
+    }
+  }
+
+  case class ListDescriptor(id: Int, tpe: Type, iter: Tree => Tree, elem: UDTDescriptor)
+    extends UDTDescriptor {
+    override def canBeKey = false
+    override def flatten = this +: elem.flatten
+
+    def getInnermostElem: UDTDescriptor = elem match {
+      case list: ListDescriptor => list.getInnermostElem
+      case _                    => elem
+    }
+
+    override def hashCode() = (id, tpe, elem).hashCode()
+    override def equals(that: Any) = that match {
+      case that @ ListDescriptor(thatId, thatTpe,  _, thatElem) =>
+        (id, tpe, elem).equals((thatId, thatTpe, thatElem))
+      case _ => false
+    }
+  }
+
+  case class BaseClassDescriptor(
+      id: Int, tpe: Type, override val getters: Seq[FieldAccessor], subTypes: Seq[UDTDescriptor])
+    extends UDTDescriptor {
+
+    override def flatten = this +: ((getters flatMap { _.desc.flatten }) ++ (subTypes flatMap { _.flatten }))
+    override def canBeKey = flatten forall { f => f.canBeKey }
+    
+    override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
+      case Nil => getters flatMap { g => g.desc.select(Nil) }
+      case head :: tail => getters find { _.getter.name.toString == head } match {
+        case None => Seq(None)
+        case Some(d : FieldAccessor) => d.desc.select(tail)
+      }
+    }
+  }
+
+  case class CaseClassDescriptor(
+      id: Int, tpe: Type, mutable: Boolean, ctor: Symbol, override val getters: Seq[FieldAccessor])
+    extends UDTDescriptor {
+
+    override val isPrimitiveProduct = getters.nonEmpty && getters.forall(_.desc.isPrimitiveProduct)
+
+    override def mkRoot = this.copy(getters = getters map { _.copy(isBaseField = false) })
+    override def flatten = this +: (getters flatMap { _.desc.flatten })
+    
+    override def canBeKey = flatten forall { f => f.canBeKey }
+
+    // Hack: ignore the ctorTpe, since two Type instances representing
+    // the same ctor function type don't appear to be considered equal. 
+    // Equality of the tpe and ctor fields implies equality of ctorTpe anyway.
+    override def hashCode = (id, tpe, ctor, getters).hashCode
+    override def equals(that: Any) = that match {
+      case CaseClassDescriptor(thatId, thatTpe, thatMutable, thatCtor, thatGetters) =>
+        (id, tpe, mutable, ctor, getters).equals(thatId, thatTpe, thatMutable, thatCtor, thatGetters)
+      case _ => false
+    }
+    
+    override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
+      case Nil => getters flatMap { g => g.desc.select(Nil) }
+      case head :: tail => getters find { _.getter.name.toString == head } match {
+        case None => Seq(None)
+        case Some(d : FieldAccessor) => d.desc.select(tail)
+      }
+    }
+  }
+
+  case class FieldAccessor(getter: Symbol, setter: Symbol, tpe: Type, isBaseField: Boolean, desc: UDTDescriptor)
+
+  case class RecursiveDescriptor(id: Int, tpe: Type, refId: Int) extends UDTDescriptor {
+    override def flatten = Seq(this)
+    override def canBeKey = tpe <:< typeOf[org.apache.flink.types.Key[_]]
+  }
+  
+  case class ValueDescriptor(id: Int, tpe: Type) extends UDTDescriptor {
+    override val isPrimitiveProduct = true
+    override def flatten = Seq(this)
+    override def canBeKey = tpe <:< typeOf[org.apache.flink.types.Key[_]]
+  }
+
+  case class WritableDescriptor(id: Int, tpe: Type) extends UDTDescriptor {
+    override val isPrimitiveProduct = true
+    override def flatten = Seq(this)
+    override def canBeKey = tpe <:< typeOf[org.apache.hadoop.io.WritableComparable[_]]
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
new file mode 100644
index 0000000..248c396
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.api.scala.codegen
+
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.typeutils._
+import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.types.{Value, TypeInformation}
+import org.apache.hadoop.io.Writable
+
+import scala.reflect.macros.Context
+
+private[flink] trait TypeInformationGen[C <: Context] {
+  this: MacroContextHolder[C]
+  with TypeDescriptors[C]
+  with TypeAnalyzer[C]
+  with TreeGen[C] =>
+
+  import c.universe._
+
+  // This is for external calling by TypeUtils.createTypeInfo
+  def mkTypeInfo[T: c.WeakTypeTag]: c.Expr[TypeInformation[T]] = {
+    val desc = getUDTDescriptor(weakTypeOf[T])
+    val result: c.Expr[TypeInformation[T]] = mkTypeInfo(desc)(c.WeakTypeTag(desc.tpe))
+    result
+  }
+
+  // We have this for internal use so that we can use it to recursively generate a tree of
+  // TypeInformation from a tree of UDTDescriptor
+  def mkTypeInfo[T: c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = desc match {
+    case cc@CaseClassDescriptor(_, tpe, _, _, _) =>
+      mkTupleTypeInfo(cc)(c.WeakTypeTag(tpe).asInstanceOf[c.WeakTypeTag[Product]])
+        .asInstanceOf[c.Expr[TypeInformation[T]]]
+    case p : PrimitiveDescriptor => mkPrimitiveTypeInfo(p.tpe)
+    case p : BoxedPrimitiveDescriptor => mkPrimitiveTypeInfo(p.tpe)
+    case l : ListDescriptor if l.tpe <:< typeOf[Array[_]] => mkListTypeInfo(l)
+    case v : ValueDescriptor =>
+      mkValueTypeInfo(v)(c.WeakTypeTag(v.tpe).asInstanceOf[c.WeakTypeTag[Value]])
+        .asInstanceOf[c.Expr[TypeInformation[T]]]
+    case d : WritableDescriptor =>
+      mkWritableTypeInfo(d)(c.WeakTypeTag(d.tpe).asInstanceOf[c.WeakTypeTag[Writable]])
+        .asInstanceOf[c.Expr[TypeInformation[T]]]
+    case d => mkGenericTypeInfo(d)
+  }
+
+  def mkTupleTypeInfo[T <: Product : c.WeakTypeTag](
+      desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = {
+    val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
+    val fields = desc.getters.toList map { field =>
+      mkTypeInfo(field.desc)(c.WeakTypeTag(field.tpe)).tree
+    }
+    val fieldsExpr = c.Expr[Seq[TypeInformation[_]]](mkList(fields))
+    val instance = mkCreateTupleInstance[T](desc)(c.WeakTypeTag(desc.tpe))
+    reify {
+      new ScalaTupleTypeInfo[T](tpeClazz.splice, fieldsExpr.splice) {
+        override def createSerializer: TypeSerializer[T] = {
+          val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
+          for (i <- 0 until getArity) {
+            fieldSerializers(i) = types(i).createSerializer
+          }
+
+          new ScalaTupleSerializer[T](tupleType, fieldSerializers) {
+            override def createInstance(fields: Array[AnyRef]): T = {
+              instance.splice
+            }
+          }
+        }
+      }
+    }
+  }
+
+  def mkListTypeInfo[T: c.WeakTypeTag](desc: ListDescriptor): c.Expr[TypeInformation[T]] = {
+    val arrayClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
+    val elementClazz = c.Expr[Class[T]](Literal(Constant(desc.elem.tpe)))
+    val elementTypeInfo = mkTypeInfo(desc.elem)
+    desc.elem match {
+      // special case for string, which in scala is a primitive, but not in java
+      case p: PrimitiveDescriptor if p.tpe <:< typeOf[String] =>
+        reify {
+          BasicArrayTypeInfo.getInfoFor(arrayClazz.splice)
+        }
+      case p: PrimitiveDescriptor =>
+        reify {
+          PrimitiveArrayTypeInfo.getInfoFor(arrayClazz.splice)
+        }
+      case bp: BoxedPrimitiveDescriptor =>
+        reify {
+          BasicArrayTypeInfo.getInfoFor(arrayClazz.splice)
+        }
+      case _ =>
+        reify {
+          ObjectArrayTypeInfo.getInfoFor(
+            arrayClazz.splice,
+            elementTypeInfo.splice).asInstanceOf[TypeInformation[T]]
+        }
+    }
+  }
+
+  def mkValueTypeInfo[T <: Value : c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
+    val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
+    reify {
+      new ValueTypeInfo[T](tpeClazz.splice)
+    }
+  }
+
+  def mkWritableTypeInfo[T <: Writable : c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
+    val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
+    reify {
+      new WritableTypeInfo[T](tpeClazz.splice)
+    }
+  }
+
+  def mkGenericTypeInfo[T: c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = {
+    val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe)))
+    reify {
+      TypeExtractor.createTypeInfo(tpeClazz.splice).asInstanceOf[TypeInformation[T]]
+    }
+  }
+
+  def mkPrimitiveTypeInfo[T: c.WeakTypeTag](tpe: Type): c.Expr[TypeInformation[T]] = {
+    val tpeClazz = c.Expr[Class[T]](Literal(Constant(tpe)))
+    reify {
+      BasicTypeInfo.getInfoFor(tpeClazz.splice)
+    }
+  }
+
+  def mkCreateTupleInstance[T: c.WeakTypeTag](desc: CaseClassDescriptor): c.Expr[T] = {
+    val fields = desc.getters.zipWithIndex.map { case (field, i) =>
+      val call = mkCall(Ident(newTermName("fields")), "apply")(List(Literal(Constant(i))))
+      mkAsInstanceOf(call)(c.WeakTypeTag(field.tpe))
+    }
+    val result = Apply(Select(New(TypeTree(desc.tpe)), nme.CONSTRUCTOR), fields.toList)
+    c.Expr[T](result)
+  }
+
+//    def mkCaseClassTypeInfo[T: c.WeakTypeTag](desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = {
+//      val tpeClazz = c.Expr[Class[_]](Literal(Constant(desc.tpe)))
+//      val caseFields = mkCaseFields(desc)
+//      reify {
+//        new ScalaTupleTypeInfo[T] {
+//          def createSerializer: TypeSerializer[T] = {
+//            null
+//          }
+//
+//          val fields: Map[String, TypeInformation[_]] = caseFields.splice
+//          val clazz = tpeClazz.splice
+//        }
+//      }
+//    }
+//
+//  private def mkCaseFields(desc: UDTDescriptor): c.Expr[Map[String, TypeInformation[_]]] = {
+//    val fields = getFields("_root_", desc).toList map { case (fieldName, fieldDesc) =>
+//      val nameTree = c.Expr(Literal(Constant(fieldName)))
+//      val fieldTypeInfo = mkTypeInfo(fieldDesc)(c.WeakTypeTag(fieldDesc.tpe))
+//      reify { (nameTree.splice, fieldTypeInfo.splice) }.tree
+//    }
+//
+//    c.Expr(mkMap(fields))
+//  }
+//
+//  protected def getFields(name: String, desc: UDTDescriptor): Seq[(String, UDTDescriptor)] = desc match {
+//    // Flatten product types
+//    case CaseClassDescriptor(_, _, _, _, getters) =>
+//      getters filterNot { _.isBaseField } flatMap { f => getFields(name + "." + f.getter.name, f.desc) }
+//    case _ => Seq((name, desc))
+//  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTAnalyzer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTAnalyzer.scala
deleted file mode 100644
index 2dad277..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTAnalyzer.scala
+++ /dev/null
@@ -1,344 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.Option.option2Iterable
-import scala.collection.GenTraversableOnce
-import scala.collection.mutable
-import scala.reflect.macros.Context
-import scala.util.DynamicVariable
-
-import org.apache.flink.types.BooleanValue
-import org.apache.flink.types.ByteValue
-import org.apache.flink.types.CharValue
-import org.apache.flink.types.DoubleValue
-import org.apache.flink.types.FloatValue
-import org.apache.flink.types.IntValue
-import org.apache.flink.types.StringValue
-import org.apache.flink.types.LongValue
-import org.apache.flink.types.ShortValue
-
-
-trait UDTAnalyzer[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with Loggers[C] =>
-  import c.universe._
-
-  // This value is controlled by the udtRecycling compiler option
-  var enableMutableUDTs = false
-
-  private val mutableTypes = mutable.Set[Type]()
-
-  def getUDTDescriptor(tpe: Type): UDTDescriptor = (new UDTAnalyzerInstance with Logger).analyze(tpe)
-
-  private def normTpe(tpe: Type): Type = {
-    // TODO Figure out what the heck this does
-    //      val currentThis = ThisType(localTyper.context.enclClass.owner)
-    //      currentThis.baseClasses.foldLeft(tpe map { _.dealias }) { (tpe, base) => tpe.substThis(base, currentThis) }
-    tpe
-  }
-
-  private def typeArgs(tpe: Type) = tpe match { case TypeRef(_, _, args) => args }
-
-  private class UDTAnalyzerInstance { this: Logger =>
-
-    private val cache = new UDTAnalyzerCache()
-
-    def analyze(tpe: Type): UDTDescriptor = {
-
-      val normed = normTpe(tpe)
-
-      cache.getOrElseUpdate(normed) { id =>
-        normed match {
-          case PrimitiveType(default, wrapper) => PrimitiveDescriptor(id, normed, default, wrapper)
-          case BoxedPrimitiveType(default, wrapper, box, unbox) => BoxedPrimitiveDescriptor(id, normed, default, wrapper, box, unbox)
-          case ListType(elemTpe, iter) => analyzeList(id, normed, elemTpe, iter)
-          case CaseClassType() => analyzeCaseClass(id, normed)
-          case BaseClassType() => analyzeClassHierarchy(id, normed)
-          case PactValueType() => PactValueDescriptor(id, normed)
-          case _ => UnsupportedDescriptor(id, normed, Seq("Unsupported type " + normed))
-        }
-      }
-    }
-
-    private def analyzeList(id: Int, tpe: Type, elemTpe: Type, iter: Tree => Tree): UDTDescriptor = analyze(elemTpe) match {
-      case UnsupportedDescriptor(_, _, errs) => UnsupportedDescriptor(id, tpe, errs)
-      case desc => ListDescriptor(id, tpe, iter, desc)
-    }
-
-    private def analyzeClassHierarchy(id: Int, tpe: Type): UDTDescriptor = {
-
-      val tagField = {
-        val (intTpe, intDefault, intWrapper) = PrimitiveType.intPrimitive
-        FieldAccessor(NoSymbol, NoSymbol, NullaryMethodType(intTpe), true, PrimitiveDescriptor(cache.newId, intTpe, intDefault, intWrapper))
-      }
-      
-//      c.info(c.enclosingPosition, "KNOWN SUBCLASSES: " + tpe.typeSymbol.asClass.knownDirectSubclasses.toList, true)
-
-      val subTypes = tpe.typeSymbol.asClass.knownDirectSubclasses.toList flatMap { d =>
-
-        val dTpe = // verbosely[Type] { dTpe => d.tpe + " <: " + tpe + " instantiated as " + dTpe + " (" + (if (dTpe <:< tpe) "Valid" else "Invalid") + " subtype)" } 
-          {
-            val tArgs = (tpe.typeSymbol.asClass.typeParams, typeArgs(tpe)).zipped.toMap
-            val dArgs = d.asClass.typeParams map { dp =>
-              val tArg = tArgs.keySet.find { tp => dp == tp.typeSignature.asSeenFrom(d.typeSignature, tpe.typeSymbol).typeSymbol }
-              tArg map { tArgs(_) } getOrElse dp.typeSignature
-            }
-
-            normTpe(appliedType(d.asType.toType, dArgs))
-          }
-//      c.info(c.enclosingPosition, "dTpe: " + dTpe, true)
-
-        if (dTpe <:< tpe)
-          Some(analyze(dTpe))
-        else
-          None
-      }
-
-//      c.info(c.enclosingPosition, c.enclosingRun.units.size +  " SUBTYPES: " + subTypes, true)
-
-      val errors = subTypes flatMap { _.findByType[UnsupportedDescriptor] }
-
-//      c.info(c.enclosingPosition, "ERROS: " + errors, true)
-
-      errors match {
-        case _ :: _ => UnsupportedDescriptor(id, tpe, errors flatMap { case UnsupportedDescriptor(_, subType, errs) => errs map { err => "Subtype " + subType + " - " + err } })
-        case Nil if subTypes.isEmpty => UnsupportedDescriptor(id, tpe, Seq("No instantiable subtypes found for base class"))
-        case Nil => {
-
-          val (tParams, tArgs) = tpe.typeSymbol.asClass.typeParams.zip(typeArgs(tpe)).unzip
-          val baseMembers = tpe.members filter { f => f.isMethod } filter { f => f.asMethod.isSetter } map {
-            f => (f, f.asMethod.setter, normTpe(f.asMethod.returnType))
-          }
-
-          val subMembers = subTypes map {
-            case BaseClassDescriptor(_, _, getters, _) => getters
-            case CaseClassDescriptor(_, _, _, _, getters) => getters
-            case _ => Seq()
-          }
-
-          val baseFields = baseMembers flatMap {
-            case (bGetter, bSetter, bTpe) => {
-              val accessors = subMembers map {
-                _ find { sf =>
-                  sf.getter.name == bGetter.name && sf.tpe.termSymbol.asMethod.returnType <:< bTpe.termSymbol.asMethod.returnType
-                }
-              }
-              accessors.forall { _.isDefined } match {
-                case true => Some(FieldAccessor(bGetter, bSetter, bTpe, true, analyze(bTpe.termSymbol.asMethod.returnType)))
-                case false => None
-              }
-            }
-          }
-
-          def wireBaseFields(desc: UDTDescriptor): UDTDescriptor = {
-
-            def updateField(field: FieldAccessor) = {
-              baseFields find { bf => bf.getter.name == field.getter.name } match {
-                case Some(FieldAccessor(_, _, _, _, desc)) => field.copy(isBaseField = true, desc = desc)
-                case None => field
-              }
-            }
-
-            desc match {
-              case desc @ BaseClassDescriptor(_, _, getters, subTypes) => desc.copy(getters = getters map updateField, subTypes = subTypes map wireBaseFields)
-              case desc @ CaseClassDescriptor(_, _, _, _, getters) => desc.copy(getters = getters map updateField)
-              case _ => desc
-            }
-          }
-
-          //Debug.report("BaseClass " + tpe + " has shared fields: " + (baseFields.map { m => m.sym.name + ": " + m.tpe }))
-          BaseClassDescriptor(id, tpe, tagField +: (baseFields.toSeq), subTypes map wireBaseFields)
-        }
-      }
-
-    }
-
-    private def analyzeCaseClass(id: Int, tpe: Type): UDTDescriptor = {
-
-      tpe.baseClasses exists { bc => !(bc == tpe.typeSymbol) && bc.asClass.isCaseClass } match {
-
-        case true => UnsupportedDescriptor(id, tpe, Seq("Case-to-case inheritance is not supported."))
-
-        case false => {
-
-          val ctors = tpe.declarations collect {
-            case m: MethodSymbol if m.isPrimaryConstructor => m
-          }
-
-          ctors match {
-            case c1 :: c2 :: _ => UnsupportedDescriptor(id, tpe, Seq("Multiple constructors found, this is not supported."))
-            case c :: Nil => {
-              val caseFields = c.paramss.flatten.map {
-                sym =>
-                  {
-                    val methodSym = tpe.member(sym.name).asMethod
-                    (methodSym.getter, methodSym.setter, methodSym.returnType.asSeenFrom(tpe, tpe.typeSymbol))
-                  }
-              }
-              val fields = caseFields map {
-                case (fgetter, fsetter, fTpe) => FieldAccessor(fgetter, fsetter, fTpe, false, analyze(fTpe))
-              }
-              val mutable = maybeVerbosely[Boolean](m => m && mutableTypes.add(tpe))(_ => "Detected recyclable type: " + tpe) {
-                enableMutableUDTs && (fields forall { f => f.setter != NoSymbol })
-              }
-              fields filter { _.desc.isInstanceOf[UnsupportedDescriptor] } match {
-                case errs @ _ :: _ => {
-                  val msgs = errs flatMap { f =>
-                    (f: @unchecked) match {
-                      case FieldAccessor(fgetter, _, _, _, UnsupportedDescriptor(_, fTpe, errors)) => errors map { err => "Field " + fgetter.name + ": " + fTpe + " - " + err }
-                    }
-                  }
-                  UnsupportedDescriptor(id, tpe, msgs)
-                }
-                case Nil => CaseClassDescriptor(id, tpe, mutable, c, fields.toSeq)
-              }
-            }
-          }
-        }
-      }
-    }
-
-    private object PrimitiveType {
-
-      def intPrimitive: (Type, Literal, Type) = {
-        val (d, w) = primitives(definitions.IntClass)
-        (definitions.IntTpe, d, w)
-      }
-
-      def unapply(tpe: Type): Option[(Literal, Type)] = primitives.get(tpe.typeSymbol)
-    }
-
-    private object BoxedPrimitiveType {
-
-      def unapply(tpe: Type): Option[(Literal, Type, Tree => Tree, Tree => Tree)] = boxedPrimitives.get(tpe.typeSymbol)
-    }
-
-    private object ListType {
-
-      def unapply(tpe: Type): Option[(Type, Tree => Tree)] = tpe match {
-
-        case ArrayType(elemTpe) => {
-          val iter = { source: Tree => 
-            Select(source, "iterator": TermName)
-          }
-          Some(elemTpe, iter)
-        }
-
-        case TraversableType(elemTpe) => {
-          val iter = { source: Tree => Select(source, "toIterator": TermName) }
-          Some(elemTpe, iter)
-        }
-
-        case _ => None
-      }
-
-      private object ArrayType {
-        def unapply(tpe: Type): Option[Type] = tpe match {
-          case TypeRef(_, _, elemTpe :: Nil) if tpe <:< typeOf[Array[_]] => Some(elemTpe)
-          case _ => None
-        }
-      }
-
-      private object TraversableType {
-        def unapply(tpe: Type): Option[Type] = tpe match {
-          case _ if tpe <:< typeOf[GenTraversableOnce[_]] => {
-            //              val abstrElemTpe = genTraversableOnceClass.typeConstructor.typeParams.head.tpe
-            //              val elemTpe = abstrElemTpe.asSeenFrom(tpe, genTraversableOnceClass)
-            //              Some(elemTpe)
-            // TODO make sure this shit works as it should
-            tpe match {
-              case TypeRef(_, _, elemTpe :: Nil) => Some(elemTpe.asSeenFrom(tpe, tpe.typeSymbol))
-            }
-          }
-          case _ => None
-        }
-      }
-    }
-
-    private object CaseClassType {
-      def unapply(tpe: Type): Boolean = tpe.typeSymbol.asClass.isCaseClass
-    }
-
-    private object BaseClassType {
-      def unapply(tpe: Type): Boolean = tpe.typeSymbol.asClass.isAbstractClass && tpe.typeSymbol.asClass.isSealed
-    }
-    
-    private object PactValueType {
-      def unapply(tpe: Type): Boolean = tpe.typeSymbol.asClass.baseClasses exists { s => s.fullName == "org.apache.flink.types.Value" }
-    }
-
-    private class UDTAnalyzerCache {
-
-      private val caches = new DynamicVariable[Map[Type, RecursiveDescriptor]](Map())
-      private val idGen = new Counter
-
-      def newId = idGen.next
-
-      def getOrElseUpdate(tpe: Type)(orElse: Int => UDTDescriptor): UDTDescriptor = {
-
-        val id = idGen.next
-        val cache = caches.value
-
-        cache.get(tpe) map { _.copy(id = id) } getOrElse {
-          val ref = RecursiveDescriptor(id, tpe, id)
-          caches.withValue(cache + (tpe -> ref)) {
-            orElse(id)
-          }
-        }
-      }
-    }
-  }
-
-  lazy val primitives = Map[Symbol, (Literal, Type)](
-    definitions.BooleanClass -> (Literal(Constant(false)), typeOf[BooleanValue]),
-    definitions.ByteClass -> (Literal(Constant(0: Byte)), typeOf[ByteValue]),
-    definitions.CharClass -> (Literal(Constant(0: Char)), typeOf[CharValue]),
-    definitions.DoubleClass -> (Literal(Constant(0: Double)), typeOf[DoubleValue]),
-    definitions.FloatClass -> (Literal(Constant(0: Float)), typeOf[FloatValue]),
-    definitions.IntClass -> (Literal(Constant(0: Int)), typeOf[IntValue]),
-    definitions.LongClass -> (Literal(Constant(0: Long)), typeOf[LongValue]),
-    definitions.ShortClass -> (Literal(Constant(0: Short)), typeOf[ShortValue]),
-    definitions.StringClass -> (Literal(Constant(null: String)), typeOf[StringValue]))
-
-  lazy val boxedPrimitives = {
-
-    def getBoxInfo(prim: Symbol, primName: String, boxName: String) = {
-      val (default, wrapper) = primitives(prim)
-      val box = { t: Tree => 
-        Apply(Select(Select(Ident(newTermName("scala")), newTermName("Predef")), newTermName(primName + "2" + boxName)), List(t))
-      }
-      val unbox = { t: Tree =>
-        Apply(Select(Select(Ident(newTermName("scala")), newTermName("Predef")), newTermName(boxName + "2" + primName)), List(t))
-      }
-      (default, wrapper, box, unbox)
-    }
-
-    Map(
-      typeOf[java.lang.Boolean].typeSymbol -> getBoxInfo(definitions.BooleanClass, "boolean", "Boolean"),
-      typeOf[java.lang.Byte].typeSymbol -> getBoxInfo(definitions.ByteClass, "byte", "Byte"),
-      typeOf[java.lang.Character].typeSymbol -> getBoxInfo(definitions.CharClass, "char", "Character"),
-      typeOf[java.lang.Double].typeSymbol -> getBoxInfo(definitions.DoubleClass, "double", "Double"),
-      typeOf[java.lang.Float].typeSymbol -> getBoxInfo(definitions.FloatClass, "float", "Float"),
-      typeOf[java.lang.Integer].typeSymbol -> getBoxInfo(definitions.IntClass, "int", "Integer"),
-      typeOf[java.lang.Long].typeSymbol -> getBoxInfo(definitions.LongClass, "long", "Long"),
-      typeOf[java.lang.Short].typeSymbol -> getBoxInfo(definitions.ShortClass, "short", "Short"))
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTDescriptors.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTDescriptors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTDescriptors.scala
deleted file mode 100644
index e57e7bb..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTDescriptors.scala
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.language.postfixOps
-
-import scala.reflect.macros.Context
-import scala.reflect.classTag
-import scala.reflect.ClassTag
-import scala.Option.option2Iterable
-
-trait UDTDescriptors[C <: Context] { this: MacroContextHolder[C] => 
-  import c.universe._
-
-  abstract sealed class UDTDescriptor {
-    val id: Int
-    val tpe: Type
-    val isPrimitiveProduct: Boolean = false
-    
-    def canBeKey: Boolean
-
-    def mkRoot: UDTDescriptor = this
-
-    def flatten: Seq[UDTDescriptor]
-    def getters: Seq[FieldAccessor] = Seq()
-
-    def select(member: String): Option[UDTDescriptor] = getters find { _.getter.name.toString == member } map { _.desc }
-    
-    def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
-      case Nil => Seq(Some(this))
-      case head :: tail => getters find { _.getter.name.toString == head } match {
-        case None => Seq(None)
-        case Some(d : FieldAccessor) => d.desc.select(tail)
-      }
-    }
-
-    def findById(id: Int): Option[UDTDescriptor] = flatten.find { _.id == id }
-
-    def findByType[T <: UDTDescriptor: ClassTag]: Seq[T] = {
-      val clazz = classTag[T].runtimeClass
-      flatten filter { item => clazz.isAssignableFrom(item.getClass) } map { _.asInstanceOf[T] }
-    }
-
-    def getRecursiveRefs: Seq[UDTDescriptor] = findByType[RecursiveDescriptor] flatMap { rd => findById(rd.refId) } map { _.mkRoot } distinct
-  }
-
-  case class UnsupportedDescriptor(id: Int, tpe: Type, errors: Seq[String]) extends UDTDescriptor {
-    override def flatten = Seq(this)
-    
-    def canBeKey = false
-  }
-
-  case class PrimitiveDescriptor(id: Int, tpe: Type, default: Literal, wrapper: Type) extends UDTDescriptor {
-    override val isPrimitiveProduct = true
-    override def flatten = Seq(this)
-    override def canBeKey = wrapper <:< typeOf[org.apache.flink.types.Key[_]]
-  }
-
-  case class BoxedPrimitiveDescriptor(id: Int, tpe: Type, default: Literal, wrapper: Type, box: Tree => Tree, unbox: Tree => Tree) extends UDTDescriptor {
-
-    override val isPrimitiveProduct = true
-    override def flatten = Seq(this)
-    override def canBeKey = wrapper <:< typeOf[org.apache.flink.types.Key[_]]
-
-    override def hashCode() = (id, tpe, default, wrapper, "BoxedPrimitiveDescriptor").hashCode()
-    override def equals(that: Any) = that match {
-      case BoxedPrimitiveDescriptor(thatId, thatTpe, thatDefault, thatWrapper, _, _) => (id, tpe, default, wrapper).equals(thatId, thatTpe, thatDefault, thatWrapper)
-      case _ => false
-    }
-  }
-
-  case class ListDescriptor(id: Int, tpe: Type, iter: Tree => Tree, elem: UDTDescriptor) extends UDTDescriptor {
-    override def canBeKey = false
-    override def flatten = this +: elem.flatten
-
-    def getInnermostElem: UDTDescriptor = elem match {
-      case list: ListDescriptor => list.getInnermostElem
-      case _                    => elem
-    }
-
-    override def hashCode() = (id, tpe, elem).hashCode()
-    override def equals(that: Any) = that match {
-      case that @ ListDescriptor(thatId, thatTpe,  _, thatElem) => (id, tpe, elem).equals((thatId, thatTpe, thatElem))
-      case _ => false
-    }
-  }
-
-  case class BaseClassDescriptor(id: Int, tpe: Type, override val getters: Seq[FieldAccessor], subTypes: Seq[UDTDescriptor]) extends UDTDescriptor {
-    override def flatten = this +: ((getters flatMap { _.desc.flatten }) ++ (subTypes flatMap { _.flatten }))
-    override def canBeKey = flatten forall { f => f.canBeKey }
-    
-    override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
-      case Nil => getters flatMap { g => g.desc.select(Nil) }
-      case head :: tail => getters find { _.getter.name.toString == head } match {
-        case None => Seq(None)
-        case Some(d : FieldAccessor) => d.desc.select(tail)
-      }
-    }
-  }
-
-  case class CaseClassDescriptor(id: Int, tpe: Type, mutable: Boolean, ctor: Symbol, override val getters: Seq[FieldAccessor]) extends UDTDescriptor {
-
-    override val isPrimitiveProduct = !getters.isEmpty && getters.forall(_.desc.isPrimitiveProduct)
-
-    override def mkRoot = this.copy(getters = getters map { _.copy(isBaseField = false) })
-    override def flatten = this +: (getters flatMap { _.desc.flatten })
-    
-    override def canBeKey = flatten forall { f => f.canBeKey }
-
-    // Hack: ignore the ctorTpe, since two Type instances representing
-    // the same ctor function type don't appear to be considered equal. 
-    // Equality of the tpe and ctor fields implies equality of ctorTpe anyway.
-    override def hashCode = (id, tpe, ctor, getters).hashCode
-    override def equals(that: Any) = that match {
-      case CaseClassDescriptor(thatId, thatTpe, thatMutable, thatCtor, thatGetters) => (id, tpe, mutable, ctor, getters).equals(thatId, thatTpe, thatMutable, thatCtor, thatGetters)
-      case _ => false
-    }
-    
-    override def select(path: List[String]): Seq[Option[UDTDescriptor]] = path match {
-      case Nil => getters flatMap { g => g.desc.select(Nil) }
-      case head :: tail => getters find { _.getter.name.toString == head } match {
-        case None => Seq(None)
-        case Some(d : FieldAccessor) => d.desc.select(tail)
-      }
-    }
-  }
-
-  case class FieldAccessor(getter: Symbol, setter: Symbol, tpe: Type, isBaseField: Boolean, desc: UDTDescriptor)
-
-  case class RecursiveDescriptor(id: Int, tpe: Type, refId: Int) extends UDTDescriptor {
-    override def flatten = Seq(this)
-    override def canBeKey = tpe <:< typeOf[org.apache.flink.types.Key[_]]
-  }
-  
-  case class PactValueDescriptor(id: Int, tpe: Type) extends UDTDescriptor {
-    override val isPrimitiveProduct = true
-    override def flatten = Seq(this)
-    override def canBeKey = tpe <:< typeOf[org.apache.flink.types.Key[_]]
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTGen.scala
deleted file mode 100644
index c2293ff..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/UDTGen.scala
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import scala.reflect.macros.Context
-
-import org.apache.flink.api.scala.analysis.UDT
-
-import org.apache.flink.types.ListValue
-import org.apache.flink.types.Record
-
-trait UDTGen[C <: Context] { this: MacroContextHolder[C] with UDTDescriptors[C] with UDTAnalyzer[C] with TreeGen[C] with SerializerGen[C] with SerializeMethodGen[C] with DeserializeMethodGen[C] with Loggers[C] =>
-  import c.universe._
-
-  def mkUdtClass[T: c.WeakTypeTag](): (ClassDef, Tree) = {
-    val desc = getUDTDescriptor(weakTypeOf[T])
-
-    val udtName = c.fresh[TypeName]("GeneratedUDTDescriptor")
-    val udt = mkClass(udtName, Flag.FINAL, List(weakTypeOf[UDT[T]]), {
-      val (ser, createSer) = mkUdtSerializerClass[T](creatorName = "createSerializer")
-      val ctor = mkMethod(nme.CONSTRUCTOR.toString(), NoFlags, List(), NoType, {
-        Block(List(mkSuperCall(Nil)), mkUnit)
-      })
-
-      List(ser, createSer, ctor, mkFieldTypes(desc), mkUDTIdToIndexMap(desc))
-    })
-    
-    val (_, udtTpe) = typeCheck(udt)
-    
-    (udt, mkCtorCall(udtTpe, Nil))
-  }
-  
-  private def mkFieldTypes(desc: UDTDescriptor): Tree = {
-
-    mkVal("fieldTypes", Flag.OVERRIDE | Flag.FINAL, false, typeOf[Array[Class[_ <: org.apache.flink.types.Value]]], {
-
-      val fieldTypes = getIndexFields(desc).toList map {
-        case PrimitiveDescriptor(_, _, _, wrapper) => Literal(Constant(wrapper))
-        case BoxedPrimitiveDescriptor(_, _, _, wrapper, _, _) => Literal(Constant(wrapper))
-        case PactValueDescriptor(_, tpe) => Literal(Constant(tpe))
-        case ListDescriptor(_, _, _, _) => Literal(Constant(typeOf[ListValue[org.apache.flink.types.Value]]))
-        // Box inner instances of recursive types
-        case RecursiveDescriptor(_, _, _) => Literal(Constant(typeOf[Record]))
-        case BaseClassDescriptor(_, _, _, _) => throw new RuntimeException("Illegal descriptor for basic record field.")
-        case CaseClassDescriptor(_, _, _, _, _) => throw new RuntimeException("Illegal descriptor for basic record field.")
-        case UnsupportedDescriptor(_, _, _) => throw new RuntimeException("Illegal descriptor for basic record field.")
-      }
-      Apply(Select(Select(Ident("scala": TermName), "Array": TermName), "apply": TermName), fieldTypes)
-    })
-  }
-  
-  private def mkUDTIdToIndexMap(desc: UDTDescriptor): Tree = {
-
-    mkVal("udtIdMap", Flag.OVERRIDE | Flag.FINAL, false, typeOf[Map[Int, Int]], {
-
-      val fieldIds = getIndexFields(desc).toList map {
-        case PrimitiveDescriptor(id, _, _, _) => Literal(Constant(id))
-        case BoxedPrimitiveDescriptor(id, _, _, _, _, _) => Literal(Constant(id))
-        case ListDescriptor(id, _, _, _) => Literal(Constant(id))
-        case RecursiveDescriptor(id, _, _) => Literal(Constant(id))
-        case PactValueDescriptor(id, _) => Literal(Constant(id))
-        case BaseClassDescriptor(_, _, _, _) => throw new RuntimeException("Illegal descriptor for basic record field.")
-        case CaseClassDescriptor(_, _, _, _, _) => throw new RuntimeException("Illegal descriptor for basic record field.")
-        case UnsupportedDescriptor(_, _, _) => throw new RuntimeException("Illegal descriptor for basic record field.")
-      }
-      val fields = fieldIds.zipWithIndex map { case (id, idx) =>
-        val idExpr = c.Expr[Int](id)
-        val idxExpr = c.Expr[Int](Literal(Constant(idx)))
-        reify { (idExpr.splice, idxExpr.splice) }.tree
-      }
-      Apply(Select(Select(Select(Ident("scala": TermName), "Predef": TermName), "Map": TermName), "apply": TermName), fields)
-    })
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Util.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Util.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Util.scala
deleted file mode 100644
index 278a5e2..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/Util.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.codegen
-
-import language.experimental.macros
-import scala.reflect.macros.Context
-
-import org.apache.flink.api.scala.analysis.UDT
-
-object Util {
-  
-  implicit def createUDT[T]: UDT[T] = macro createUDTImpl[T]
-
-  def createUDTImpl[T: c.WeakTypeTag](c: Context): c.Expr[UDT[T]] = {
-    import c.universe._
-
-    val slave = MacroContextHolder.newMacroHelper(c)
-
-    val (udt, createUdt) = slave.mkUdtClass[T]
-
-    val udtResult = reify {
-      c.Expr[UDT[T]](createUdt).splice
-    }
-    
-    c.Expr[UDT[T]](Block(List(udt), udtResult.tree))
-  }
-
-  // filter out forwards that dont forward from one field position to the same field position
-  def filterNonForwards(from: Array[Int], to: Array[Int]): Array[Int] = {
-    from.zip(to).filter( z => z._1 == z._2).map { _._1}
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
new file mode 100644
index 0000000..5218745
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala
+
+import org.apache.commons.lang3.Validate
+import org.apache.flink.api.common.functions.{RichCrossFunction, CrossFunction}
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.Collector
+
+import scala.reflect.ClassTag
+
+/**
+ * A specific [[DataSet]] that results from a `cross` operation. The result of a default cross is a
+ * tuple containing the two values from the two sides of the cartesian product. The result of the
+ * cross can be changed by specifying a custom cross function using the `apply` method or by
+ * providing a [[RichCrossFunction]].
+ *
+ * Example:
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val crossResult = left.cross(right) {
+ *     (left, right) => new MyCrossResult(left, right)
+ *   }
+ * }}}
+ *
+ * @tparam T Type of the left input of the cross.
+ * @tparam O Type of the right input of the cross.
+ */
+trait CrossDataSet[T, O] extends DataSet[(T, O)] {
+
+  /**
+   * Creates a new [[DataSet]] where the result for each pair of elements is the result
+   * of the given function.
+   */
+  def apply[R: TypeInformation: ClassTag](fun: (T, O) => R): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing each pair of values to the given function.
+   * The function can output zero or more elements using the [[Collector]] which will form the
+   * result.
+   *
+   * A [[RichCrossFunction]] can be used to access the
+   * broadcast variables and the [[org.apache.flink.api.common.functions.RuntimeContext]].
+   */
+  def apply[R: TypeInformation: ClassTag](joiner: CrossFunction[T, O, R]): DataSet[R]
+}
+
+/**
+ * Private implementation for [[CrossDataSet]] to keep the implementation details, i.e. the
+ * parameters of the constructor, hidden.
+ */
+private[flink] class CrossDataSetImpl[T, O](
+    crossOperator: CrossOperator[T, O, (T, O)],
+    thisSet: JavaDataSet[T],
+    otherSet: JavaDataSet[O])
+  extends DataSet(crossOperator)
+  with CrossDataSet[T, O] {
+
+  def apply[R: TypeInformation: ClassTag](fun: (T, O) => R): DataSet[R] = {
+    Validate.notNull(fun, "Cross function must not be null.")
+    val crosser = new CrossFunction[T, O, R] {
+      def cross(left: T, right: O): R = {
+        fun(left, right)
+      }
+    }
+    val crossOperator = new CrossOperator[T, O, R](
+      thisSet,
+      otherSet,
+      crosser,
+      implicitly[TypeInformation[R]])
+    wrap(crossOperator)
+  }
+
+  def apply[R: TypeInformation: ClassTag](crosser: CrossFunction[T, O, R]): DataSet[R] = {
+    Validate.notNull(crosser, "Cross function must not be null.")
+    val crossOperator = new CrossOperator[T, O, R](
+      thisSet,
+      otherSet,
+      crosser,
+      implicitly[TypeInformation[R]])
+    wrap(crossOperator)
+  }
+}
+
+private[flink] object CrossDataSetImpl {
+  def createCrossOperator[T, O](leftSet: JavaDataSet[T], rightSet: JavaDataSet[O]) = {
+    val crosser = new CrossFunction[T, O, (T, O)] {
+      def cross(left: T, right: O) = {
+        (left, right)
+      }
+    }
+    val returnType = new ScalaTupleTypeInfo[(T, O)](
+      classOf[(T, O)], Seq(leftSet.getType, rightSet.getType)) {
+
+      override def createSerializer: TypeSerializer[(T, O)] = {
+        val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
+        for (i <- 0 until getArity) {
+          fieldSerializers(i) = types(i).createSerializer
+        }
+
+        new ScalaTupleSerializer[(T, O)](classOf[(T, O)], fieldSerializers) {
+          override def createInstance(fields: Array[AnyRef]) = {
+            (fields(0).asInstanceOf[T], fields(1).asInstanceOf[O])
+          }
+        }
+      }
+    }
+    val crossOperator = new CrossOperator[T, O, (T, O)](leftSet, rightSet, crosser, returnType)
+
+    new CrossDataSetImpl(crossOperator, leftSet, rightSet)
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala
deleted file mode 100644
index 6c7e93b..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CoGroupFunction.scala
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.scala.functions
-
-import java.util.{Iterator => JIterator}
-
-import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT}
-import org.apache.flink.api.scala.analysis.UDF2
-
-import org.apache.flink.api.java.record.functions.{CoGroupFunction => JCoGroupFunction}
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-import org.apache.flink.configuration.Configuration
-
-
-abstract class CoGroupFunctionBase[LeftIn: UDT, RightIn: UDT, Out: UDT] extends JCoGroupFunction with Serializable {
-  val leftInputUDT = implicitly[UDT[LeftIn]]
-  val rightInputUDT = implicitly[UDT[RightIn]]
-  val outputUDT = implicitly[UDT[Out]]
-  val udf: UDF2[LeftIn, RightIn, Out] = new UDF2(leftInputUDT, rightInputUDT, outputUDT)
-
-  protected val outputRecord = new Record()
-
-  protected lazy val leftIterator: DeserializingIterator[LeftIn] = new DeserializingIterator(udf.getLeftInputDeserializer)
-  protected lazy val leftForwardFrom: Array[Int] = udf.getLeftForwardIndexArrayFrom
-  protected lazy val leftForwardTo: Array[Int] = udf.getLeftForwardIndexArrayTo
-  protected lazy val rightIterator: DeserializingIterator[RightIn] = new DeserializingIterator(udf.getRightInputDeserializer)
-  protected lazy val rightForwardFrom: Array[Int] = udf.getRightForwardIndexArrayFrom
-  protected lazy val rightForwardTo: Array[Int] = udf.getRightForwardIndexArrayTo
-  protected lazy val serializer: UDTSerializer[Out] = udf.getOutputSerializer
-
-  override def open(config: Configuration) = {
-    super.open(config)
-
-    this.outputRecord.setNumFields(udf.getOutputLength)
-  }
-}
-
-abstract class CoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGroupFunctionBase[LeftIn, RightIn, Out] with Function2[Iterator[LeftIn], Iterator[RightIn], Out] {
-  override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = {
-    val firstLeftRecord = leftIterator.initialize(leftRecords)
-    val firstRightRecord = rightIterator.initialize(rightRecords)
-
-    if (firstRightRecord != null) {
-      outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo)
-    }
-    if (firstLeftRecord != null) {
-      outputRecord.copyFrom(firstLeftRecord, leftForwardFrom, leftForwardTo)
-    }
-
-    val output = apply(leftIterator, rightIterator)
-
-    serializer.serialize(output, outputRecord)
-    out.collect(outputRecord)
-  }
-}
-
-abstract class FlatCoGroupFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CoGroupFunctionBase[LeftIn, RightIn, Out] with Function2[Iterator[LeftIn], Iterator[RightIn], Iterator[Out]] {
-  override def coGroup(leftRecords: JIterator[Record], rightRecords: JIterator[Record], out: Collector[Record]) = {
-    val firstLeftRecord = leftIterator.initialize(leftRecords)
-    outputRecord.copyFrom(firstLeftRecord, leftForwardFrom, leftForwardTo)
-
-    val firstRightRecord = rightIterator.initialize(rightRecords)
-    outputRecord.copyFrom(firstRightRecord, rightForwardFrom, rightForwardTo)
-
-    val output = apply(leftIterator, rightIterator)
-
-    if (output.nonEmpty) {
-
-      for (item <- output) {
-        serializer.serialize(item, outputRecord)
-        out.collect(outputRecord)
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala
deleted file mode 100644
index c292100..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/CrossFunction.scala
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.functions
-
-import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT}
-import org.apache.flink.api.scala.analysis.UDF2
-
-import org.apache.flink.api.java.record.functions.{CrossFunction => JCrossFunction}
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-
-abstract class CrossFunctionBase[LeftIn: UDT, RightIn: UDT, Out: UDT] extends JCrossFunction with Serializable {
-  val leftInputUDT = implicitly[UDT[LeftIn]]
-  val rightInputUDT = implicitly[UDT[RightIn]]
-  val outputUDT = implicitly[UDT[Out]]
-  val udf: UDF2[LeftIn, RightIn, Out] = new UDF2(leftInputUDT, rightInputUDT, outputUDT)
-
-  protected lazy val leftDeserializer: UDTSerializer[LeftIn] = udf.getLeftInputDeserializer
-  protected lazy val leftForwardFrom: Array[Int] = udf.getLeftForwardIndexArrayFrom
-  protected lazy val leftForwardTo: Array[Int] = udf.getLeftForwardIndexArrayTo
-  protected lazy val leftDiscard: Array[Int] = udf.getLeftDiscardIndexArray.filter(_ < udf.getOutputLength)
-  protected lazy val rightDeserializer: UDTSerializer[RightIn] = udf.getRightInputDeserializer
-  protected lazy val rightForwardFrom: Array[Int] = udf.getRightForwardIndexArrayFrom
-  protected lazy val rightForwardTo: Array[Int] = udf.getRightForwardIndexArrayTo
-  protected lazy val serializer: UDTSerializer[Out] = udf.getOutputSerializer
-  protected lazy val outputLength: Int = udf.getOutputLength
-
-}
-
-abstract class CrossFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends CrossFunctionBase[LeftIn, RightIn, Out] with Function2[LeftIn, RightIn, Out] {
-  override def cross(leftRecord: Record, rightRecord: Record) : Record = {
-    val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-    val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-    val output = apply(left, right)
-
-    leftRecord.setNumFields(outputLength)
-
-    for (field <- leftDiscard)
-      leftRecord.setNull(field)
-
-    leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo)
-    leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo)
-
-    serializer.serialize(output, leftRecord)
-    leftRecord
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/DeserializingIterator.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/DeserializingIterator.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/DeserializingIterator.scala
deleted file mode 100644
index 0d5c128..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/DeserializingIterator.scala
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.functions
-
-import java.util.{ Iterator => JIterator }
-
-import org.apache.flink.api.scala.analysis.UDTSerializer
-
-import org.apache.flink.types.Record
-
-protected final class DeserializingIterator[T](deserializer: UDTSerializer[T]) extends Iterator[T] {
-
-  private var source: JIterator[Record] = null
-  private var first: Record = null
-  private var fresh = true
-
-  final def initialize(records: JIterator[Record]): Record = {
-    source = records
-
-    if (source.hasNext) {
-      fresh = true
-      first = source.next()
-    } else {
-      fresh = false
-      first = null
-    }
-    
-    first
-  }
-
-  final def hasNext = fresh || source.hasNext
-
-  final def next(): T = {
-
-    if (fresh) {
-      fresh = false
-      val record = deserializer.deserializeRecyclingOff(first)
-      first = null
-      record
-    } else {
-      deserializer.deserializeRecyclingOff(source.next())
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/JoinFunction.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/JoinFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/JoinFunction.scala
deleted file mode 100644
index a1d2e2b..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/JoinFunction.scala
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.functions
-
-import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT}
-import org.apache.flink.api.scala.analysis.UDF2
-
-import org.apache.flink.api.java.record.functions.{JoinFunction => JJoinFunction}
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-
-abstract class JoinFunctionBase[LeftIn: UDT, RightIn: UDT, Out: UDT] extends JJoinFunction with Serializable {
-  val leftInputUDT = implicitly[UDT[LeftIn]]
-  val rightInputUDT = implicitly[UDT[RightIn]]
-  val outputUDT = implicitly[UDT[Out]]
-  val udf: UDF2[LeftIn, RightIn, Out] = new UDF2(leftInputUDT, rightInputUDT, outputUDT)
-
-  protected lazy val leftDeserializer: UDTSerializer[LeftIn] = udf.getLeftInputDeserializer
-  protected lazy val leftDiscard: Array[Int] = udf.getLeftDiscardIndexArray.filter(_ < udf.getOutputLength)
-  protected lazy val leftForwardFrom: Array[Int] = udf.getLeftForwardIndexArrayFrom
-  protected lazy val leftForwardTo: Array[Int] = udf.getLeftForwardIndexArrayTo
-  protected lazy val rightDeserializer: UDTSerializer[RightIn] = udf.getRightInputDeserializer
-  protected lazy val rightForwardFrom: Array[Int] = udf.getRightForwardIndexArrayFrom
-  protected lazy val rightForwardTo: Array[Int] = udf.getRightForwardIndexArrayTo
-  protected lazy val serializer: UDTSerializer[Out] = udf.getOutputSerializer
-  protected lazy val outputLength: Int = udf.getOutputLength
-}
-
-abstract class JoinFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends JoinFunctionBase[LeftIn, RightIn, Out] with Function2[LeftIn, RightIn, Out] {
-  override def join(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = {
-    val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-    val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-    val output = apply(left, right)
-
-    leftRecord.setNumFields(outputLength)
-    for (field <- leftDiscard)
-      leftRecord.setNull(field)
-
-    leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo)
-    leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo)
-
-    serializer.serialize(output, leftRecord)
-    out.collect(leftRecord)
-  }
-}
-
-abstract class FlatJoinFunction[LeftIn: UDT, RightIn: UDT, Out: UDT] extends JoinFunctionBase[LeftIn, RightIn, Out] with Function2[LeftIn, RightIn, Iterator[Out]] {
-  override def join(leftRecord: Record, rightRecord: Record, out: Collector[Record]) = {
-    val left = leftDeserializer.deserializeRecyclingOn(leftRecord)
-    val right = rightDeserializer.deserializeRecyclingOn(rightRecord)
-    val output = apply(left, right)
-
-    if (output.nonEmpty) {
-
-      leftRecord.setNumFields(outputLength)
-
-      for (field <- leftDiscard)
-        leftRecord.setNull(field)
-
-      leftRecord.copyFrom(rightRecord, rightForwardFrom, rightForwardTo)
-      leftRecord.copyFrom(leftRecord, leftForwardFrom, leftForwardTo)
-
-      for (item <- output) {
-        serializer.serialize(item, leftRecord)
-        out.collect(leftRecord)
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/MapFunction.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/MapFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/MapFunction.scala
deleted file mode 100644
index 445d443..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/MapFunction.scala
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.functions
-
-import org.apache.flink.api.scala.analysis.{UDTSerializer, UDT}
-import org.apache.flink.api.scala.analysis.UDF1
-
-import org.apache.flink.api.java.record.functions.{MapFunction => JMapFunction}
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-
-abstract class MapFunctionBase[In: UDT, Out: UDT] extends JMapFunction with Serializable{
-  val inputUDT: UDT[In] = implicitly[UDT[In]]
-  val outputUDT: UDT[Out] = implicitly[UDT[Out]]
-  val udf: UDF1[In, Out] = new UDF1(inputUDT, outputUDT)
-
-  protected lazy val deserializer: UDTSerializer[In] = udf.getInputDeserializer
-  protected lazy val serializer: UDTSerializer[Out] = udf.getOutputSerializer
-  protected lazy val discard: Array[Int] = udf.getDiscardIndexArray
-  protected lazy val outputLength: Int = udf.getOutputLength
-}
-
-abstract class MapFunction[In: UDT, Out: UDT] extends MapFunctionBase[In, Out] with Function1[In, Out] {
-  override def map(record: Record, out: Collector[Record]) = {
-    val input = deserializer.deserializeRecyclingOn(record)
-    val output = apply(input)
-
-    record.setNumFields(outputLength)
-
-    for (field <- discard)
-      record.setNull(field)
-
-    serializer.serialize(output, record)
-    out.collect(record)
-  }
-}
-
-abstract class FlatMapFunction[In: UDT, Out: UDT] extends MapFunctionBase[In, Out] with Function1[In, Iterator[Out]] {
-  override def map(record: Record, out: Collector[Record]) = {
-    val input = deserializer.deserializeRecyclingOn(record)
-    val output = apply(input)
-
-    if (output.nonEmpty) {
-
-      record.setNumFields(outputLength)
-
-      for (field <- discard)
-        record.setNull(field)
-
-      for (item <- output) {
-
-        serializer.serialize(item, record)
-        out.collect(record)
-      }
-    }
-  }
-}
-
-abstract class FilterFunction[In: UDT, Out: UDT] extends MapFunctionBase[In, Out] with Function1[In, Boolean]  {
-  override def map(record: Record, out: Collector[Record]) = {
-    val input = deserializer.deserializeRecyclingOn(record)
-    if (apply(input)) {
-      out.collect(record)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala
deleted file mode 100644
index de7b7d1..0000000
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/functions/ReduceFunction.scala
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.scala.functions
-
-import scala.Iterator
-
-import java.util.{Iterator => JIterator}
-
-import org.apache.flink.api.scala.analysis.{UDTSerializer, FieldSelector, UDT}
-import org.apache.flink.api.scala.analysis.UDF1
-
-import org.apache.flink.api.java.record.functions.{ReduceFunction => JReduceFunction}
-import org.apache.flink.types.Record
-import org.apache.flink.util.Collector
-
-
-
-abstract class ReduceFunctionBase[In: UDT, Out: UDT] extends JReduceFunction with Serializable {
-  val inputUDT: UDT[In] = implicitly[UDT[In]]
-  val outputUDT: UDT[Out] = implicitly[UDT[Out]]
-  val udf: UDF1[In, Out] = new UDF1(inputUDT, outputUDT)
-
-  protected val reduceRecord = new Record()
-
-  protected lazy val reduceIterator: DeserializingIterator[In] = new DeserializingIterator(udf.getInputDeserializer)
-  protected lazy val reduceSerializer: UDTSerializer[Out] = udf.getOutputSerializer
-  protected lazy val reduceForwardFrom: Array[Int] = udf.getForwardIndexArrayFrom
-  protected lazy val reduceForwardTo: Array[Int] = udf.getForwardIndexArrayTo
-}
-
-abstract class ReduceFunction[In: UDT] extends ReduceFunctionBase[In, In] with Function2[In, In, In] {
-
-  override def combine(records: JIterator[Record], out: Collector[Record]) = {
-    reduce(records, out)
-  }
-
-  override def reduce(records: JIterator[Record], out: Collector[Record]) = {
-    val firstRecord = reduceIterator.initialize(records)
-    reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-    val output = reduceIterator.reduce(apply)
-
-    reduceSerializer.serialize(output, reduceRecord)
-    out.collect(reduceRecord)
-  }
-}
-
-abstract class GroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase[In, Out] with Function1[Iterator[In], Out] {
-  override def reduce(records: JIterator[Record], out: Collector[Record]) = {
-    val firstRecord = reduceIterator.initialize(records)
-    reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-    val output = apply(reduceIterator)
-
-    reduceSerializer.serialize(output, reduceRecord)
-    out.collect(reduceRecord)
-  }
-}
-
-abstract class CombinableGroupReduceFunction[In: UDT, Out: UDT] extends ReduceFunctionBase[In, Out] with Function1[Iterator[In], Out] {
-  override def combine(records: JIterator[Record], out: Collector[Record]) = {
-    val firstRecord = reduceIterator.initialize(records)
-    reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-    val output = combine(reduceIterator)
-
-    reduceSerializer.serialize(output, reduceRecord)
-    out.collect(reduceRecord)
-  }
-
-  override def reduce(records: JIterator[Record], out: Collector[Record]) = {
-    val firstRecord = reduceIterator.initialize(records)
-    reduceRecord.copyFrom(firstRecord, reduceForwardFrom, reduceForwardTo)
-
-    val output = reduce(reduceIterator)
-
-    reduceSerializer.serialize(output, reduceRecord)
-    out.collect(reduceRecord)
-  }
-
-  def reduce(records: Iterator[In]): Out
-  def combine(records: Iterator[In]): Out
-
-  def apply(record: Iterator[In]): Out = throw new RuntimeException("This should never be called.")
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
new file mode 100644
index 0000000..8d24ee1
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala
+
+import org.apache.commons.lang3.Validate
+import org.apache.flink.api.common.InvalidProgramException
+import org.apache.flink.api.common.functions.{JoinFunction, RichFlatJoinFunction, FlatJoinFunction}
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin.WrappingFlatJoinFunction
+import org.apache.flink.api.java.operators.JoinOperator.{EquiJoin, JoinHint}
+import org.apache.flink.api.java.operators._
+import org.apache.flink.api.java.{DataSet => JavaDataSet}
+import org.apache.flink.api.scala.typeutils.{ScalaTupleSerializer, ScalaTupleTypeInfo}
+import org.apache.flink.types.TypeInformation
+import org.apache.flink.util.Collector
+
+import scala.reflect.ClassTag
+
+/**
+ * A specific [[DataSet]] that results from a `join` operation. The result of a default join is a
+ * tuple containing the two values from the two sides of the join. The result of the join can be
+ * changed by specifying a custom join function using the `apply` method or by providing a
+ * [[RichFlatJoinFunction]].
+ *
+ * Example:
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val joinResult = left.join(right).where(0, 2).isEqualTo(0, 1) {
+ *     (left, right) => new MyJoinResult(left, right)
+ *   }
+ * }}}
+ *
+ * Or, using key selector functions with tuple data types:
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val joinResult = left.join(right).where({_._1}).isEqualTo({_._1) {
+ *     (left, right) => new MyJoinResult(left, right)
+ *   }
+ * }}}
+ *
+ * @tparam T Type of the left input of the join.
+ * @tparam O Type of the right input of the join.
+ */
+trait JoinDataSet[T, O] extends DataSet[(T, O)] {
+
+  /**
+   * Creates a new [[DataSet]] where the result for each pair of joined elements is the result
+   * of the given function. You can either return an element or choose to return [[None]],
+   * which allows implementing a filter directly in the join function.
+   */
+  def apply[R: TypeInformation: ClassTag](fun: (T, O) => Option[R]): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing each pair of joined values to the given function.
+   * The function can output zero or more elements using the [[Collector]] which will form the
+   * result.
+   */
+  def apply[R: TypeInformation: ClassTag](fun: (T, O, Collector[R]) => Unit): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing each pair of joined values to the given function.
+   * The function can output zero or more elements using the [[Collector]] which will form the
+   * result.
+   *
+   * A [[RichFlatJoinFunction]] can be used to access the
+   * broadcast variables and the [[org.apache.flink.api.common.functions.RuntimeContext]].
+   */
+  def apply[R: TypeInformation: ClassTag](joiner: FlatJoinFunction[T, O, R]): DataSet[R]
+
+  /**
+   * Creates a new [[DataSet]] by passing each pair of joined values to the given function.
+   * The function must output one value. The concatenation of those will be new the DataSet.
+   *
+   * A [[org.apache.flink.api.common.functions.RichJoinFunction]] can be used to access the
+   * broadcast variables and the [[org.apache.flink.api.common.functions.RuntimeContext]].
+   */
+  def apply[R: TypeInformation: ClassTag](joiner: JoinFunction[T, O, R]): DataSet[R]
+}
+
+/**
+ * Private implementation for [[JoinDataSet]] to keep the implementation details, i.e. the
+ * parameters of the constructor, hidden.
+ */
+private[flink] class JoinDataSetImpl[T, O](
+    joinOperator: EquiJoin[T, O, (T, O)],
+    thisSet: JavaDataSet[T],
+    otherSet: JavaDataSet[O],
+    thisKeys: Keys[T],
+    otherKeys: Keys[O])
+  extends DataSet(joinOperator)
+  with JoinDataSet[T, O] {
+
+  def apply[R: TypeInformation: ClassTag](fun: (T, O) => Option[R]): DataSet[R] = {
+    Validate.notNull(fun, "Join function must not be null.")
+    val joiner = new FlatJoinFunction[T, O, R] {
+      def join(left: T, right: O, out: Collector[R]) = {
+        fun(left, right) map { out.collect(_) }
+      }
+    }
+    val joinOperator = new EquiJoin[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, joiner, implicitly[TypeInformation[R]], JoinHint.OPTIMIZER_CHOOSES)
+    wrap(joinOperator)
+  }
+
+  def apply[R: TypeInformation: ClassTag](fun: (T, O, Collector[R]) => Unit): DataSet[R] = {
+    Validate.notNull(fun, "Join function must not be null.")
+    val joiner = new FlatJoinFunction[T, O, R] {
+      def join(left: T, right: O, out: Collector[R]) = {
+        fun(left, right, out)
+      }
+    }
+    val joinOperator = new EquiJoin[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, joiner, implicitly[TypeInformation[R]], JoinHint.OPTIMIZER_CHOOSES)
+    wrap(joinOperator)
+  }
+
+  def apply[R: TypeInformation: ClassTag](joiner: FlatJoinFunction[T, O, R]): DataSet[R] = {
+    Validate.notNull(joiner, "Join function must not be null.")
+    val joinOperator = new EquiJoin[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, joiner, implicitly[TypeInformation[R]], JoinHint.OPTIMIZER_CHOOSES)
+    wrap(joinOperator)
+  }
+
+  def apply[R: TypeInformation: ClassTag](fun: JoinFunction[T, O, R]): DataSet[R] = {
+    Validate.notNull(fun, "Join function must not be null.")
+
+    val generatedFunction: FlatJoinFunction[T, O, R] = new WrappingFlatJoinFunction[T, O, R](fun)
+
+    val joinOperator = new EquiJoin[T, O, R](thisSet, otherSet, thisKeys,
+      otherKeys, generatedFunction, implicitly[TypeInformation[R]], JoinHint.OPTIMIZER_CHOOSES)
+    wrap(joinOperator)
+  }
+}
+
+/**
+ * An unfinished join operation that results from [[DataSet.join()]] The keys for the left and right
+ * side must be specified using first `where` and then `isEqualTo`. For example:
+ *
+ * {{{
+ *   val left = ...
+ *   val right = ...
+ *   val joinResult = left.join(right).where(...).isEqualTo(...)
+ * }}}
+ * @tparam T The type of the left input of the join.
+ * @tparam O The type of the right input of the join.
+ */
+trait UnfinishedJoinOperation[T, O] extends UnfinishedKeyPairOperation[T, O, JoinDataSet[T, O]]
+
+/**
+ * Private implementation for [[UnfinishedJoinOperation]] to keep the implementation details,
+ * i.e. the parameters of the constructor, hidden.
+ */
+private[flink] class UnfinishedJoinOperationImpl[T, O](
+    leftSet: JavaDataSet[T],
+    rightSet: JavaDataSet[O],
+    joinHint: JoinHint)
+  extends UnfinishedKeyPairOperation[T, O, JoinDataSet[T, O]](leftSet, rightSet)
+  with UnfinishedJoinOperation[T, O] {
+
+  private[flink] def finish(leftKey: Keys[T], rightKey: Keys[O]) = {
+    val joiner = new FlatJoinFunction[T, O, (T, O)] {
+      def join(left: T, right: O, out: Collector[(T, O)]) = {
+        out.collect((left, right))
+      }
+    }
+    val returnType = new ScalaTupleTypeInfo[(T, O)](
+      classOf[(T, O)], Seq(leftSet.getType, rightSet.getType)) {
+
+      override def createSerializer: TypeSerializer[(T, O)] = {
+        val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity)
+        for (i <- 0 until getArity()) {
+          fieldSerializers(i) = types(i).createSerializer
+        }
+
+        new ScalaTupleSerializer[(T, O)](classOf[(T, O)], fieldSerializers) {
+          override def createInstance(fields: Array[AnyRef]) = {
+            (fields(0).asInstanceOf[T], fields(1).asInstanceOf[O])
+          }
+        }
+      }
+    }
+    val joinOperator = new EquiJoin[T, O, (T, O)](
+      leftSet, rightSet, leftKey, rightKey, joiner, returnType, joinHint)
+
+    // sanity check solution set key mismatches
+    leftSet match {
+      case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
+        leftKey match {
+          case keyFields: Keys.FieldPositionKeys[_] =>
+            val positions: Array[Int] = keyFields.computeLogicalKeyPositions
+            solutionSet.checkJoinKeyFields(positions)
+          case _ =>
+            throw new InvalidProgramException("Currently, the solution set may only be joined " +
+              "with " +
+              "using tuple field positions.")
+        }
+      case _ =>
+    }
+    rightSet match {
+      case solutionSet: DeltaIteration.SolutionSetPlaceHolder[_] =>
+        rightKey match {
+          case keyFields: Keys.FieldPositionKeys[_] =>
+            val positions: Array[Int] = keyFields.computeLogicalKeyPositions
+            solutionSet.checkJoinKeyFields(positions)
+          case _ =>
+            throw new InvalidProgramException("Currently, the solution set may only be joined " +
+              "with " +
+              "using tuple field positions.")
+        }
+      case _ =>
+    }
+
+    new JoinDataSetImpl(joinOperator, leftSet, rightSet, leftKey, rightKey)
+  }
+}
\ No newline at end of file


[47/60] [doc] Switch parser to kramdown, normalize Headings

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/java_api_transformations.md
----------------------------------------------------------------------
diff --git a/docs/java_api_transformations.md b/docs/java_api_transformations.md
deleted file mode 100644
index c1dcc19..0000000
--- a/docs/java_api_transformations.md
+++ /dev/null
@@ -1,677 +0,0 @@
----
-title: "Java API Transformations"
----
-
-<section id="top">
-DataSet Transformations
------------------------
-
-This document gives a deep-dive into the available transformations on DataSets. For a general introduction to the
-Flink Java API, please refer to the [API guide](java_api_guide.html)
-
-
-### Map
-
-The Map transformation applies a user-defined `MapFunction` on each element of a DataSet.
-It implements a one-to-one mapping, that is, exactly one element must be returned by
-the function.
-
-The following code transforms a `DataSet` of Integer pairs into a `DataSet` of Integers:
-
-```java
-// MapFunction that adds two integer values
-public class IntAdder implements MapFunction<Tuple2<Integer, Integer>, Integer> {
-  @Override
-  public Integer map(Tuple2<Integer, Integer> in) {
-    return in.f0 + in.f1;
-  }
-}
-
-// [...]
-DataSet<Tuple2<Integer, Integer>> intPairs = // [...]
-DataSet<Integer> intSums = intPairs.map(new IntAdder());
-```
-
-### FlatMap
-
-The FlatMap transformation applies a user-defined `FlatMapFunction` on each element of a `DataSet`.
-This variant of a map function can return arbitrary many result elements (including none) for each input element.
-
-The following code transforms a `DataSet` of text lines into a `DataSet` of words:
-
-```java
-// FlatMapFunction that tokenizes a String by whitespace characters and emits all String tokens.
-public class Tokenizer implements FlatMapFunction<String, String> {
-  @Override
-  public void flatMap(String value, Collector<String> out) {
-    for (String token : value.split("\\W")) {
-      out.collect(token);
-    }
-  }
-}
-
-// [...]
-DataSet<String> textLines = // [...]
-DataSet<String> words = textLines.flatMap(new Tokenizer());
-```
-
-### MapPartition
-
-The MapPartition function transforms a parallel partition in a single function call. The function get the partition as an `Iterable` stream and
-can produce an arbitrary number of result values. The number of elements in each partition depends on the degree-of-parallelism
-and previous operations.
-
-The following code transforms a `DataSet` of text lines into a `DataSet` of counts per partition:
-
-```java
-public class PartitionCounter implements MapPartitionFunction<String, Long> {
-
-  public void mapPartition(Iterable<String> values, Collector<Long> out) {
-    long c = 0;
-    for (String s : values) {
-      c++;
-    }
-    out.collect(c);
-  }
-}
-
-// [...]
-DataSet<String> textLines = // [...]
-DataSet<Long> counts = textLines.mapPartition(new PartitionCounter());
-```
-
-### Filter
-
-The Filter transformation applies a user-defined `FilterFunction` on each element of a `DataSet` and retains only those elements for which the function returns `true`.
-
-The following code removes all Integers smaller than zero from a `DataSet`:
-
-```java
-// FilterFunction that filters out all Integers smaller than zero.
-public class NaturalNumberFilter implements FilterFunction<Integer> {
-  @Override
-  public boolean filter(Integer number) {
-    return number >= 0;
-  }
-}
-
-// [...]
-DataSet<Integer> intNumbers = // [...]
-DataSet<Integer> naturalNumbers = intNumbers.filter(new NaturalNumberFilter());
-```
-
-### Project (Tuple DataSets only)
-
-The Project transformation removes or moves `Tuple` fields of a `Tuple` `DataSet`.
-The `project(int...)` method selects `Tuple` fields that should be retained by their index and defines their order in the output `Tuple`.
-The `types(Class<?> ...)`method must give the types of the output `Tuple` fields.
-
-Projections do not require the definition of a user function.
-
-The following code shows different ways to apply a Project transformation on a `DataSet`:
-
-```java
-DataSet<Tuple3<Integer, Double, String>> in = // [...]
-// converts Tuple3<Integer, Double, String> into Tuple2<String, Integer>
-DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
-```
-
-### Transformations on grouped DataSet
-
-The reduce operations can operate on grouped data sets. Specifying the key to
-be used for grouping can be done in two ways:
-
-- a `KeySelector` function or
-- one or more field position keys (`Tuple` `DataSet` only).
-
-Please look at the reduce examples to see how the grouping keys are specified.
-
-### Reduce on grouped DataSet
-
-A Reduce transformation that is applied on a grouped `DataSet` reduces each group to a single element using a user-defined `ReduceFunction`.
-For each group of input elements, a `ReduceFunction` successively combines pairs of elements into one element until only a single element for each group remains.
-
-#### Reduce on DataSet grouped by KeySelector Function
-
-A `KeySelector` function extracts a key value from each element of a `DataSet`. The extracted key value is used to group the `DataSet`.
-The following code shows how to group a POJO `DataSet` using a `KeySelector` function and to reduce it with a `ReduceFunction`.
-
-```java
-// some ordinary POJO
-public class WC {
-  public String word;
-  public int count;
-  // [...]
-}
-
-// ReduceFunction that sums Integer attributes of a POJO
-public class WordCounter implements ReduceFunction<WC> {
-  @Override
-  public WC reduce(WC in1, WC in2) {
-    return new WC(in1.word, in1.count + in2.count);
-  }
-}
-
-// [...]
-DataSet<WC> words = // [...]
-DataSet<WC> wordCounts = words
-                         // DataSet grouping with inline-defined KeySelector function
-                         .groupBy(
-                           new KeySelector<WC, String>() {
-                             public String getKey(WC wc) { return wc.word; }
-                           })
-                         // apply ReduceFunction on grouped DataSet
-                         .reduce(new WordCounter());
-```
-
-#### Reduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
-
-Field position keys specify one or more fields of a `Tuple` `DataSet` that are used as grouping keys.
-The following code shows how to use field position keys and apply a `ReduceFunction`.
-
-```java
-DataSet<Tuple3<String, Integer, Double>> tuples = // [...]
-DataSet<Tuple3<String, Integer, Double>> reducedTuples =
-                                         tuples
-                                         // group DataSet on first and second field of Tuple
-                                         .groupBy(0,1)
-                                         // apply ReduceFunction on grouped DataSet
-                                         .reduce(new MyTupleReducer());
-```
-
-### GroupReduce on grouped DataSet
-
-A GroupReduce transformation that is applied on a grouped `DataSet` calls a user-defined `GroupReduceFunction` for each group. The difference
-between this and `Reduce` is that the user defined function gets the whole group at once.
-The function is invoked with an Iterable over all elements of a group and can return an arbitrary number of result elements using the collector.
-
-#### GroupReduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
-
-The following code shows how duplicate strings can be removed from a `DataSet` grouped by Integer.
-
-```java
-public class DistinctReduce
-         implements GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
-
-  @Override
-  public void reduce(Iterable<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
-
-    Set<String> uniqStrings = new HashSet<String>();
-    Integer key = null;
-  
-    // add all strings of the group to the set
-    for (Tuple2<Integer, String> t : in) {
-      key = t.f0;
-      uniqStrings.add(t.f1);
-    }
-
-    // emit all unique strings.
-    for (String s : uniqStrings) {
-      out.collect(new Tuple2<Integer, String>(key, s));
-    }
-  }
-}
-
-// [...]
-DataSet<Tuple2<Integer, String>> input = // [...]
-DataSet<Tuple2<Integer, String>> output = input
-                           .groupBy(0)            // group DataSet by the first tuple field
-                           .reduceGroup(new DistinctReduce());  // apply GroupReduceFunction
-```
-
-#### GroupReduce on DataSet grouped by KeySelector Function
-
-Works analogous to `KeySelector` functions in Reduce transformations.
-
-#### GroupReduce on sorted groups (Tuple DataSets only)
-
-A `GroupReduceFunction` accesses the elements of a group using an Iterable. Optionally, the Iterable can hand out the elements of a group in a specified order. In many cases this can help to reduce the complexity of a user-defined `GroupReduceFunction` and improve its efficiency.
-Right now, this feature is only available for DataSets of Tuples.
-
-The following code shows another example how to remove duplicate Strings in a `DataSet` grouped by an Integer and sorted by String.
-
-```java
-// GroupReduceFunction that removes consecutive identical elements
-public class DistinctReduce
-         implements GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
-
-  @Override
-  public void reduce(Iterable<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
-    Integer key = null;
-    String comp = null;
-
-    for (Tuple2<Integer, String> t : in) {
-      key = t.f0;
-      String next = t.f1;
-
-      // check if strings are different
-      if (com == null || !next.equals(comp)) {
-        out.collect(new Tuple2<Integer, String>(key, next));
-        comp = next;
-      }
-    }
-  }
-}
-
-// [...]
-DataSet<Tuple2<Integer, String>> input = // [...]
-DataSet<Double> output = input
-                         .groupBy(0)                         // group DataSet by first field
-                         .sortGroup(1, Order.ASCENDING)      // sort groups on second tuple field
-                         .reduceGroup(new DistinctReduce());
-```
-
-**Note:** A GroupSort often comes for free if the grouping is established using a sort-based execution strategy of an operator before the reduce operation.
-
-#### Combinable GroupReduceFunctions
-
-In contrast to a `ReduceFunction`, a `GroupReduceFunction` is not
-necessarily combinable. In order to make a `GroupReduceFunction`
-combinable, you need to use the `RichGroupReduceFunction` variant,
-implement (override) the `combine()` method, and annotate the
-`GroupReduceFunction` with the `@Combinable` annotation as shown here:
-
-```java
-// Combinable GroupReduceFunction that computes two sums.
-// Note that we use the RichGroupReduceFunction because it defines the combine method
-@Combinable
-public class MyCombinableGroupReducer
-         extends RichGroupReduceFunction<Tuple3<String, Integer, Double>,
-                                     Tuple3<String, Integer, Double>> {
-  @Override
-  public void reduce(Iterable<Tuple3<String, Integer, Double>> in,
-                     Collector<Tuple3<String, Integer, Double>> out) {
-
-    String key = null
-    int intSum = 0;
-    double doubleSum = 0.0;
-
-    for (Tuple3<String, Integer, Double> curr : in) {
-      key = curr.f0;
-      intSum += curr.f1;
-      doubleSum += curr.f2;
-    }
-    // emit a tuple with both sums
-    out.collect(new Tuple3<String, Integer, Double>(key, intSum, doubleSum));
-  }
-
-  @Override
-  public void combine(Iterable<Tuple3<String, Integer, Double>> in,
-                      Collector<Tuple3<String, Integer, Double>> out)) {
-    // in some cases combine() calls can simply be forwarded to reduce().
-    this.reduce(in, out);
-  }
-}
-```
-
-### Aggregate on grouped Tuple DataSet
-
-There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
-
-- Sum,
-- Min, and
-- Max.
-
-The Aggregate transformation can only be applied on a `Tuple` `DataSet` and supports only field positions keys for grouping.
-
-The following code shows how to apply an Aggregation transformation on a `DataSet` grouped by field position keys:
-
-```java
-DataSet<Tuple3<Integer, String, Double>> input = // [...]
-DataSet<Tuple3<Integer, String, Double>> output = input
-                                   .groupBy(1)        // group DataSet on second field
-                                   .aggregate(SUM, 0) // compute sum of the first field
-                                   .and(MIN, 2);      // compute minimum of the third field
-```
-
-To apply multiple aggregations on a DataSet it is necessary to use the `.and()` function after the first aggregate, that means `.aggregate(SUM, 0).and(MIN, 2)` produces the sum of field 0 and the minimum of field 2 of the original DataSet. 
-In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggregation on an aggregation. In the given example it would produce the minimum of field 2 after calculating the sum of field 0 grouped by field 1.
-
-**Note:** The set of aggregation functions will be extended in the future.
-
-### Reduce on full DataSet
-
-The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a `DataSet`.
-The `ReduceFunction` subsequently combines pairs of elements into one element until only a single element remains.
-
-The following code shows how to sum all elements of an Integer `DataSet`:
-
-```java
-// ReduceFunction that sums Integers
-public class IntSummer implements ReduceFunction<Integer> {
-  @Override
-  public Integer reduce(Integer num1, Integer num2) {
-    return num1 + num2;
-  }
-}
-
-// [...]
-DataSet<Integer> intNumbers = // [...]
-DataSet<Integer> sum = intNumbers.reduce(new IntSummer());
-```
-
-Reducing a full `DataSet` using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
-
-### GroupReduce on full DataSet
-
-The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a `DataSet`.
-A `GroupReduceFunction` can iterate over all elements of `DataSet` and return an arbitrary number of result elements.
-
-The following example shows how to apply a GroupReduce transformation on a full `DataSet`:
-
-```java
-DataSet<Integer> input = // [...]
-// apply a (preferably combinable) GroupReduceFunction to a DataSet
-DataSet<Double> output = input.reduceGroup(new MyGroupReducer());
-```
-
-**Note:** A GroupReduce transformation on a full `DataSet` cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
-
-### Aggregate on full Tuple DataSet
-
-There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
-
-- Sum,
-- Min, and
-- Max.
-
-The Aggregate transformation can only be applied on a `Tuple` `DataSet`.
-
-The following code shows how to apply an Aggregation transformation on a full `DataSet`:
-
-```java
-DataSet<Tuple2<Integer, Double>> input = // [...]
-DataSet<Tuple2<Integer, Double>> output = input
-                                     .aggregate(SUM, 0)    // compute sum of the first field
-                                     .and(MIN, 1);    // compute minimum of the second field
-```
-
-**Note:** Extending the set of supported aggregation functions is on our roadmap.
-
-### Join
-
-The Join transformation joins two `DataSet`s into one `DataSet`. The elements of both `DataSet`s are joined on one or more keys which can be specified using
-
-- a `KeySelector` function or
-- one or more field position keys (`Tuple` `DataSet` only).
-
-There are a few different ways to perform a Join transformation which are shown in the following.
-
-#### Default Join (Join into Tuple2)
-
-The default Join transformation produces a new `Tuple``DataSet` with two fields. Each tuple holds a joined element of the first input `DataSet` in the first tuple field and a matching element of the second input `DataSet` in the second field.
-
-The following code shows a default Join transformation using field position keys:
-
-```java
-DataSet<Tuple2<Integer, String>> input1 = // [...]
-DataSet<Tuple2<Double, Integer>> input2 = // [...]
-// result dataset is typed as Tuple2
-DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Double, Integer>>>
-            result = input1.join(input2)
-                           .where(0)       // key of the first input
-                           .equalTo(1);    // key of the second input
-```
-
-#### Join with JoinFunction
-
-A Join transformation can also call a user-defined `JoinFunction` to process joining tuples.
-A `JoinFunction` receives one element of the first input `DataSet` and one element of the second input `DataSet` and returns exactly one element.
-
-The following code performs a join of `DataSet` with custom java objects and a `Tuple` `DataSet` using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
-
-```java
-// some POJO
-public class Rating {
-  public String name;
-  public String category;
-  public int points;
-}
-
-// Join function that joins a custom POJO with a Tuple
-public class PointWeighter
-         implements JoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
-
-  @Override
-  public Tuple2<String, Double> join(Rating rating, Tuple2<String, Double> weight) {
-    // multiply the points and rating and construct a new output tuple
-    return new Tuple2<String, Double>(rating.name, rating.points * weight.f1);
-  }
-}
-
-DataSet<Rating> ratings = // [...]
-DataSet<Tuple2<String, Double>> weights = // [...]
-DataSet<Tuple2<String, Double>>
-            weightedRatings =
-            ratings.join(weights)
-
-                   // key of the first input
-                   .where(new KeySelection<Rating, String>() {
-                            public String getKey(Rating r) { return r.category; }
-                          })
-
-                   // key of the second input
-                   .equalTo(new KeySelection<Tuple2<String, Double>, String>() {
-                              public String getKey(Tuple2<String, Double> t) { return t.f0; }
-                            })
-
-                   // applying the JoinFunction on joining pairs
-                   .with(new PointWeighter());
-```
-
-#### Join with FlatJoinFunction
-
-Analogous to Map and FlatMap, a FlatJoin function behaves in the same
-way as a JoinFunction, but instead of returning one element, it can
-return (collect), zero, one, or more elements.
-{% highlight java %}
-public class PointWeighter
-         implements FlatJoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
-  @Override
-  public void join(Rating rating, Tuple2<String, Double> weight,
-	  Collector<Tuple2<String, Double>> out) {
-	if (weight.f1 > 0.1) {
-		out.collect(new Tuple2<String, Double>(rating.name, rating.points * weight.f1));
-	}
-  }
-}
-
-DataSet<Tuple2<String, Double>>
-            weightedRatings =
-            ratings.join(weights) // [...]
-{% endhighlight %}
-
-#### Join with Projection
-
-A Join transformation can construct result tuples using a projection as shown here:
-
-```java
-DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
-DataSet<Tuple2<Integer, Double>> input2 = // [...]
-DataSet<Tuple4<Integer, String, Double, Byte>
-            result =
-            input1.join(input2)
-                  // key definition on first DataSet using a field position key
-                  .where(0)
-                  // key definition of second DataSet using a field position key
-                  .equalTo(0)
-                  // select and reorder fields of matching tuples
-                  .projectFirst(0,2).projectSecond(1).projectFirst(1)
-                  .types(Integer.class, String.class, Double.class, Byte.class);
-```
-
-`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output `Tuple`. The order of indexes defines the order of fields in the output tuple.
-The join projection works also for non-`Tuple` `DataSet`s. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output `Tuple`.
-
-#### Join with DataSet Size Hint
-
-In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to join as shown here:
-
-```java
-DataSet<Tuple2<Integer, String>> input1 = // [...]
-DataSet<Tuple2<Integer, String>> input2 = // [...]
-
-DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
-            result1 =
-            // hint that the second DataSet is very small
-            input1.joinWithTiny(input2)
-                  .where(0)
-                  .equalTo(0);
-
-DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
-            result2 =
-            // hint that the second DataSet is very large
-            input1.joinWithHuge(input2)
-                  .where(0)
-                  .equalTo(0);
-```
-
-### Cross
-
-The Cross transformation combines two `DataSet`s into one `DataSet`. It builds all pairwise combinations of the elements of both input `DataSet`s, i.e., it builds a Cartesian product.
-The Cross transformation either calls a user-defined `CrossFunction` on each pair of elements or applies a projection. Both modes are shown in the following.
-
-**Note:** Cross is potentially a *very* compute-intensive operation which can challenge even large compute clusters!
-
-#### Cross with User-Defined Function
-
-A Cross transformation can call a user-defined `CrossFunction`. A `CrossFunction` receives one element of the first input and one element of the second input and returns exactly one result element.
-
-The following code shows how to apply a Cross transformation on two `DataSet`s using a `CrossFunction`:
-
-```java
-public class Coord {
-  public int id;
-  public int x;
-  public int y;
-}
-
-// CrossFunction computes the Euclidean distance between two Coord objects.
-public class EuclideanDistComputer
-         implements CrossFunction<Coord, Coord, Tuple3<Integer, Integer, Double>> {
-
-  @Override
-  public Tuple3<Integer, Integer, Double> cross(Coord c1, Coord c2) {
-    // compute Euclidean distance of coordinates
-    double dist = sqrt(pow(c1.x - c2.x, 2) + pow(c1.y - c2.y, 2));
-    return new Tuple3<Integer, Integer, Double>(c1.id, c2.id, dist);
-  }
-}
-
-DataSet<Coord> coords1 = // [...]
-DataSet<Coord> coords2 = // [...]
-DataSet<Tuple3<Integer, Integer, Double>>
-            distances =
-            coords1.cross(coords2)
-                   // apply CrossFunction
-                   .with(new EuclideanDistComputer());
-```
-
-#### Cross with Projection
-
-A Cross transformation can also construct result tuples using a projection as shown here:
-
-```java
-DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
-DataSet<Tuple2<Integer, Double>> input2 = // [...]
-DataSet<Tuple4<Integer, Byte, Integer, Double>
-            result =
-            input1.cross(input2)
-                  // select and reorder fields of matching tuples
-                  .projectSecond(0).projectFirst(1,0).projectSecond(1)
-                  .types(Integer.class, Byte.class, Integer.class, Double.class);
-```
-
-The field selection in a Cross projection works the same way as in the projection of Join results.
-
-#### Cross with DataSet Size Hint
-
-In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to cross as shown here:
-
-```java
-DataSet<Tuple2<Integer, String>> input1 = // [...]
-DataSet<Tuple2<Integer, String>> input2 = // [...]
-
-DataSet<Tuple4<Integer, String, Integer, String>>
-            udfResult =
-                  // hint that the second DataSet is very small
-            input1.crossWithTiny(input2)
-                  // apply any Cross function (or projection)
-                  .with(new MyCrosser());
-
-DataSet<Tuple3<Integer, Integer, String>>
-            projectResult =
-                  // hint that the second DataSet is very large
-            input1.crossWithHuge(input2)
-                  // apply a projection (or any Cross function)
-                  .projectFirst(0,1).projectSecond(1).types(Integer.class, String.class, String.class)
-```
-
-### CoGroup
-
-The CoGroup transformation jointly processes groups of two `DataSet`s. Both `DataSet`s are grouped on a defined key and groups of both `DataSet`s that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one `DataSet` has a group, the `CoGroupFunction` is called with this group and an empty group.
-A `CoGroupFunction` can separately iterate over the elements of both groups and return an arbitrary number of result elements.
-
-Similar to Reduce, GroupReduce, and Join, keys can be defined using
-
-- a `KeySelector` function or
-- one or more field position keys (`Tuple` `DataSet` only).
-
-#### CoGroup on DataSets grouped by Field Position Keys (Tuple DataSets only)
-
-```java
-// Some CoGroupFunction definition
-class MyCoGrouper
-         implements CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Double>, Double> {
-
-  @Override
-  public void coGroup(Iterable<Tuple2<String, Integer>> iVals,
-                      Iterable<Tuple2<String, Double>> dVals,
-                      Collector<Double> out) {
-
-    Set<Integer> ints = new HashSet<Integer>();
-
-    // add all Integer values in group to set
-    for (Tuple2<String, Integer>> val : iVale) {
-      ints.add(val.f1);
-    }
-
-    // multiply each Double value with each unique Integer values of group
-    for (Tuple2<String, Double> val : dVals) {
-      for (Integer i : ints) {
-        out.collect(val.f1 * i);
-      }
-    }
-  }
-}
-
-// [...]
-DataSet<Tuple2<String, Integer>> iVals = // [...]
-DataSet<Tuple2<String, Double>> dVals = // [...]
-DataSet<Double> output = iVals.coGroup(dVals)
-                         // group first DataSet on first tuple field
-                         .where(0)
-                         // group second DataSet on first tuple field
-                         .equalTo(0)
-                         // apply CoGroup function on each pair of groups
-                         .with(new MyCoGrouper());
-```
-
-#### CoGroup on DataSets grouped by Key Selector Function
-
-Works analogous to key selector functions in Join transformations.
-
-### Union
-
-Produces the union of two `DataSet`s, which have to be of the same type. A union of more than two `DataSet`s can be implemented with multiple union calls, as shown here:
-
-```java
-DataSet<Tuple2<String, Integer>> vals1 = // [...]
-DataSet<Tuple2<String, Integer>> vals2 = // [...]
-DataSet<Tuple2<String, Integer>> vals3 = // [...]
-DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2)
-                    .union(vals3);
-```
-
-
-[Back to top](#top)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/js/codetabs.js
----------------------------------------------------------------------
diff --git a/docs/js/codetabs.js b/docs/js/codetabs.js
new file mode 100644
index 0000000..878aa32
--- /dev/null
+++ b/docs/js/codetabs.js
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/* Note: This file is originally from the Apache Spark project. */
+
+/* Custom JavaScript code in the MarkDown docs */
+
+// Enable language-specific code tabs
+function codeTabs() {
+  var counter = 0;
+  var langImages = {
+    "scala": "img/scala-sm.png",
+    "python": "img/python-sm.png",
+    "java": "img/java-sm.png"
+  };
+  $("div.codetabs").each(function() {
+    $(this).addClass("tab-content");
+
+    // Insert the tab bar
+    var tabBar = $('<ul class="nav nav-tabs" data-tabs="tabs"></ul>');
+    $(this).before(tabBar);
+
+    // Add each code sample to the tab bar:
+    var codeSamples = $(this).children("div");
+    codeSamples.each(function() {
+      $(this).addClass("tab-pane");
+      var lang = $(this).data("lang");
+      var image = $(this).data("image");
+      var notabs = $(this).data("notabs");
+      var capitalizedLang = lang.substr(0, 1).toUpperCase() + lang.substr(1);
+      var id = "tab_" + lang + "_" + counter;
+      $(this).attr("id", id);
+      if (image != null && langImages[lang]) {
+        var buttonLabel = "<img src='" +langImages[lang] + "' alt='" + capitalizedLang + "' />";
+      } else if (notabs == null) {
+        var buttonLabel = "<b>" + capitalizedLang + "</b>";
+      } else {
+        var buttonLabel = ""
+      }
+      tabBar.append(
+        '<li><a class="tab_' + lang + '" href="#' + id + '">' + buttonLabel + '</a></li>'
+      );
+    });
+
+    codeSamples.first().addClass("active");
+    tabBar.children("li").first().addClass("active");
+    counter++;
+  });
+  $("ul.nav-tabs a").click(function (e) {
+    // Toggling a tab should switch all tabs corresponding to the same language
+    // while retaining the scroll position
+    e.preventDefault();
+    var scrollOffset = $(this).offset().top - $(document).scrollTop();
+    $("." + $(this).attr('class')).tab('show');
+    $(document).scrollTop($(this).offset().top - scrollOffset);
+  });
+}
+
+function makeCollapsable(elt, accordionClass, accordionBodyId, title) {
+  $(elt).addClass("accordion-inner");
+  $(elt).wrap('<div class="accordion ' + accordionClass + '"></div>')
+  $(elt).wrap('<div class="accordion-group"></div>')
+  $(elt).wrap('<div id="' + accordionBodyId + '" class="accordion-body collapse"></div>')
+  $(elt).parent().before(
+    '<div class="accordion-heading">' +
+      '<a class="accordion-toggle" data-toggle="collapse" href="#' + accordionBodyId + '">' +
+             title +
+      '</a>' +
+    '</div>'
+  );
+}
+
+// Enable "view solution" sections (for exercises)
+function viewSolution() {
+  var counter = 0
+  $("div.solution").each(function() {
+    var id = "solution_" + counter
+    makeCollapsable(this, "", id,
+      '<i class="icon-ok-sign" style="text-decoration: none; color: #0088cc">' +
+      '</i>' + "View Solution");
+    counter++;
+  });
+}
+
+// A script to fix internal hash links because we have an overlapping top bar.
+// Based on https://github.com/twitter/bootstrap/issues/193#issuecomment-2281510
+function maybeScrollToHash() {
+  console.log("HERE");
+  if (window.location.hash && $(window.location.hash).length) {
+    console.log("HERE2", $(window.location.hash), $(window.location.hash).offset().top);
+    var newTop = $(window.location.hash).offset().top - 57;
+    $(window).scrollTop(newTop);
+  }
+}
+
+$(function() {
+  codeTabs();
+  viewSolution();
+
+  $(window).bind('hashchange', function() {
+    maybeScrollToHash();
+  });
+
+  // Scroll now too in case we had opened the page on a hash, but wait a bit because some browsers
+  // will try to do *their* initial scroll after running the onReady handler.
+  $(window).load(function() { setTimeout(function() { maybeScrollToHash(); }, 25); }); 
+});

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/local_execution.md
----------------------------------------------------------------------
diff --git a/docs/local_execution.md b/docs/local_execution.md
index 4a126bb..48d9ae7 100644
--- a/docs/local_execution.md
+++ b/docs/local_execution.md
@@ -2,7 +2,7 @@
 title:  "Local Execution"
 ---
 
-# Local Execution/Debugging
+## Local Execution/Debugging
 
 Flink can run on a single machine, even in a single Java Virtual Machine. This allows users to test and debug Flink programs locally. This section gives an overview of the local execution mechanisms.
 
@@ -17,19 +17,19 @@ The `JobExecutionResult` object, which is returned after the execution finished,
 *Note:* The local execution environments do not start any web frontend to monitor the execution.
 
 
-# Maven Dependency
+## Maven Dependency
 
 If you are developing your program in a Maven project, you have to add the `flink-clients` module using this dependency:
 
-```xml
+~~~xml
 <dependency>
   <groupId>org.apache.flink</groupId>
   <artifactId>flink-clients</artifactId>
   <version>{{site.FLINK_VERSION_STABLE}}</version>
 </dependency>
-```
+~~~
 
-# Local Environment
+## Local Environment
 
 The `LocalEnvironment` is a handle to local execution for Flink programs. Use it to run a program within a local JVM - standalone or embedded in other programs.
 
@@ -37,7 +37,7 @@ The local environment is instantiated via the method `ExecutionEnvironment.creat
 
 In most cases, calling `ExecutionEnvironment.getExecutionEnvironment()` is the even better way to go. That method returns a `LocalEnvironment` when the program is started locally (outside the command line interface), and it returns a pre-configured environment for cluster execution, when the program is invoked by the [command line interface](cli.html).
 
-```java
+~~~java
 public static void main(String[] args) throws Exception {
     ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
 
@@ -53,16 +53,16 @@ public static void main(String[] args) throws Exception {
 
     env.execute();
 }
-```
+~~~
 
 
-# Local Executor
+## Local Executor
 
 The *LocalExecutor* is similar to the local environment, but it takes a *Plan* object, which describes the program as a single executable unit. The *LocalExecutor* is typically used with the Scala API. 
 
 The following code shows how you would use the `LocalExecutor` with the Wordcount example for Scala Programs:
 
-```scala
+~~~scala
 public static void main(String[] args) throws Exception {
     val input = TextFile("hdfs://path/to/file")
 
@@ -74,15 +74,15 @@ public static void main(String[] args) throws Exception {
     val plan = new ScalaPlan(Seq(output), "Word Count")
     LocalExecutor.executePlan(p);
 }
-```
+~~~
 
 
-# LocalDistributedExecutor
+## LocalDistributedExecutor
 
 Flink also offers a `LocalDistributedExecutor` which starts multiple TaskManagers within one JVM. The standard `LocalExecutor` starts one JobManager and one TaskManager in one JVM.
 With the `LocalDistributedExecutor` you can define the number of TaskManagers to start. This is useful for debugging network related code and more of a developer tool than a user tool.
 
-```java
+~~~java
 public static void main(String[] args) throws Exception {
     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -101,6 +101,6 @@ public static void main(String[] args) throws Exception {
     lde.startNephele(2); // start two TaskManagers
     lde.run(p);
 }
-```
+~~~
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4ddc3f72/docs/local_setup.md
----------------------------------------------------------------------
diff --git a/docs/local_setup.md b/docs/local_setup.md
index d12401b..58fd847 100644
--- a/docs/local_setup.md
+++ b/docs/local_setup.md
@@ -2,62 +2,64 @@
 title:  "Local Setup"
 ---
 
+* This will be replaced by the TOC
+{:toc}
+
 This documentation is intended to provide instructions on how to run Flink locally on a single machine.
 
-# Download
+## Download
 
 Go to the [downloads page]({{site.baseurl}}/downloads/) and get the ready to run package. If you want to interact with Hadoop (e.g. HDFS or HBase), make sure to pick the Flink package **matching your Hadoop version**. When in doubt or you plan to just work with the local file system pick the package for Hadoop 1.2.x.
 
-# Requirements
+## Requirements
 
-Flink runs on **Linux**, **Mac OS X** and **Windows**. The only requirement for a local setup is **Java 1.6.x** or higher. The following manual assumes a *UNIX-like environment*, for Windows see [Flink on Windows](#windows).
+Flink runs on **Linux**, **Mac OS X** and **Windows**. The only requirement for a local setup is **Java 1.6.x** or higher. The following manual assumes a *UNIX-like environment*, for Windows see [Flink on Windows](#flink-on-windows).
 
 You can check the correct installation of Java by issuing the following command:
 
-```bash
+~~~bash
 java -version
-```
+~~~
 
 The command should output something comparable to the following:
 
-```bash
+~~~bash
 java version "1.6.0_22"
 Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
 Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)
-```
+~~~
 
-# Configuration
+## Configuration
 
 **For local mode Flink is ready to go out of the box and you don't need to change the default configuration.**
 
 The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in `conf/flink-conf.yaml` if you want to manually override the Java runtime to use. Consult the [configuration page](config.html) for further details about configuring Flink.
 
-# Starting Flink
+## Starting Flink
 
 **You are now ready to start Flink.** Unpack the downloaded archive and change to the newly created `flink` directory. There you can start Flink in local mode:
 
-```bash
+~~~bash
 $ tar xzf flink-*.tgz
 $ cd flink
 $ bin/start-local.sh
 Starting job manager
-```
+~~~
 
 You can check that the system is running by checking the log files in the `logs` directory:
 
-```bash
+~~~bash
 $ tail log/flink-*-jobmanager-*.log
 INFO ... - Initializing memory manager with 409 megabytes of memory
 INFO ... - Trying to load org.apache.flinknephele.jobmanager.scheduler.local.LocalScheduler as scheduler
 INFO ... - Setting up web info server, using web-root directory ...
 INFO ... - Web info server will display information about nephele job-manager on localhost, port 8081.
 INFO ... - Starting web info server for JobManager on port 8081
-```
+~~~
 
 The JobManager will also start a web frontend on port 8081, which you can check with your browser at `http://localhost:8081`.
 
-<section id="windows">
-# Flink on Windows
+## Flink on Windows
 
 If you want to run Flink on Windows you need to download, unpack and configure the Flink archive as mentioned above. After that you can either use the **Windows Batch** file (`.bat`) or use **Cygwin**  to run the Flink Jobmanager.
 
@@ -67,13 +69,13 @@ To start Flink in local mode from the *Windows Batch*, open the command window,
 
 Note: The ``bin`` folder of your Java Runtime Environment must be included in Window's ``%PATH%`` variable. Follow this [guide](http://www.java.com/en/download/help/path.xml) to add Java to the ``%PATH%`` variable.
 
-```bash
+~~~bash
 $ cd flink
 $ cd bin
 $ start-local.bat
 Starting Flink job manager. Webinterface by default on http://localhost:8081/.
 Do not close this batch window. Stop job manager by pressing Ctrl+C.
-```
+~~~
 
 After that, you need to open a second terminal to run jobs using `flink.bat`.
 
@@ -81,19 +83,19 @@ After that, you need to open a second terminal to run jobs using `flink.bat`.
 
 With *Cygwin* you need to start the Cygwin Terminal, navigate to your Flink directory and run the `start-local.sh` script:
 
-```bash
+~~~bash
 $ cd flink
 $ bin/start-local.sh
 Starting Nephele job manager
-```
+~~~
 
 ### Installing Flink from Git
 
 If you are installing Flink from the git repository and you are using the Windows git shell, Cygwin can produce a failure similiar to this one:
 
-```bash
+~~~bash
 c:/flink/bin/start-local.sh: line 30: $'\r': command not found
-```
+~~~
 
 This error occurs, because git is automatically transforming UNIX line endings to Windows style line endings when running in Windows. The problem is, that Cygwin can only deal with UNIX style line endings. The solution is to adjust the Cygwin settings to deal with the correct line endings by following these three steps:
 
@@ -101,17 +103,17 @@ This error occurs, because git is automatically transforming UNIX line endings t
 
 2. Determine your home directory by entering
 
-```bash
+~~~bash
 cd; pwd
-```
+~~~
 
 It will return a path under the Cygwin root path.
 
 2.  Using NotePad, WordPad or a different text editor open the file `.bash_profile` in the home directory and append the following: (If the file does not exist you have to create it)
 
-```bash
+~~~bash
 export SHELLOPTS
 set -o igncr
-```
+~~~
 
 Save the file and open a new bash shell.
\ No newline at end of file


[54/60] Turn Documentation into standalone website, add Overview Page

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/css/bootstrap.css
----------------------------------------------------------------------
diff --git a/docs/css/bootstrap.css b/docs/css/bootstrap.css
new file mode 100644
index 0000000..7f36651
--- /dev/null
+++ b/docs/css/bootstrap.css
@@ -0,0 +1,5785 @@
+/*!
+ * Bootstrap v3.1.1 (http://getbootstrap.com)
+ * Copyright 2011-2014 Twitter, Inc.
+ * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
+ */
+
+/*! normalize.css v3.0.0 | MIT License | git.io/normalize */
+html {
+  font-family: sans-serif;
+  -webkit-text-size-adjust: 100%;
+      -ms-text-size-adjust: 100%;
+}
+body {
+  margin: 0;
+}
+article,
+aside,
+details,
+figcaption,
+figure,
+footer,
+header,
+hgroup,
+main,
+nav,
+section,
+summary {
+  display: block;
+}
+audio,
+canvas,
+progress,
+video {
+  display: inline-block;
+  vertical-align: baseline;
+}
+audio:not([controls]) {
+  display: none;
+  height: 0;
+}
+[hidden],
+template {
+  display: none;
+}
+a {
+  background: transparent;
+}
+a:active,
+a:hover {
+  outline: 0;
+}
+abbr[title] {
+  border-bottom: 1px dotted;
+}
+b,
+strong {
+  font-weight: bold;
+}
+dfn {
+  font-style: italic;
+}
+h1 {
+  margin: .67em 0;
+  font-size: 2em;
+}
+mark {
+  color: #000;
+  background: #ff0;
+}
+small {
+  font-size: 80%;
+}
+sub,
+sup {
+  position: relative;
+  font-size: 75%;
+  line-height: 0;
+  vertical-align: baseline;
+}
+sup {
+  top: -.5em;
+}
+sub {
+  bottom: -.25em;
+}
+img {
+  border: 0;
+}
+svg:not(:root) {
+  overflow: hidden;
+}
+figure {
+  margin: 1em 40px;
+}
+hr {
+  height: 0;
+  -moz-box-sizing: content-box;
+       box-sizing: content-box;
+}
+pre {
+  overflow: auto;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: monospace, monospace;
+  font-size: 1em;
+}
+button,
+input,
+optgroup,
+select,
+textarea {
+  margin: 0;
+  font: inherit;
+  color: inherit;
+}
+button {
+  overflow: visible;
+}
+button,
+select {
+  text-transform: none;
+}
+button,
+html input[type="button"],
+input[type="reset"],
+input[type="submit"] {
+  -webkit-appearance: button;
+  cursor: pointer;
+}
+button[disabled],
+html input[disabled] {
+  cursor: default;
+}
+button::-moz-focus-inner,
+input::-moz-focus-inner {
+  padding: 0;
+  border: 0;
+}
+input {
+  line-height: normal;
+}
+input[type="checkbox"],
+input[type="radio"] {
+  box-sizing: border-box;
+  padding: 0;
+}
+input[type="number"]::-webkit-inner-spin-button,
+input[type="number"]::-webkit-outer-spin-button {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-box-sizing: content-box;
+     -moz-box-sizing: content-box;
+          box-sizing: content-box;
+  -webkit-appearance: textfield;
+}
+input[type="search"]::-webkit-search-cancel-button,
+input[type="search"]::-webkit-search-decoration {
+  -webkit-appearance: none;
+}
+fieldset {
+  padding: .35em .625em .75em;
+  margin: 0 2px;
+  border: 1px solid #c0c0c0;
+}
+legend {
+  padding: 0;
+  border: 0;
+}
+textarea {
+  overflow: auto;
+}
+optgroup {
+  font-weight: bold;
+}
+table {
+  border-spacing: 0;
+  border-collapse: collapse;
+}
+td,
+th {
+  padding: 0;
+}
+@media print {
+  * {
+    color: #000 !important;
+    text-shadow: none !important;
+    background: transparent !important;
+    box-shadow: none !important;
+  }
+  a,
+  a:visited {
+    text-decoration: underline;
+  }
+  a[href]:after {
+    content: " (" attr(href) ")";
+  }
+  abbr[title]:after {
+    content: " (" attr(title) ")";
+  }
+  a[href^="javascript:"]:after,
+  a[href^="#"]:after {
+    content: "";
+  }
+  pre,
+  blockquote {
+    border: 1px solid #999;
+
+    page-break-inside: avoid;
+  }
+  thead {
+    display: table-header-group;
+  }
+  tr,
+  img {
+    page-break-inside: avoid;
+  }
+  img {
+    max-width: 100% !important;
+  }
+  p,
+  h2,
+  h3 {
+    orphans: 3;
+    widows: 3;
+  }
+  h2,
+  h3 {
+    page-break-after: avoid;
+  }
+  select {
+    background: #fff !important;
+  }
+  .navbar {
+    display: none;
+  }
+  .table td,
+  .table th {
+    background-color: #fff !important;
+  }
+  .btn > .caret,
+  .dropup > .btn > .caret {
+    border-top-color: #000 !important;
+  }
+  .label {
+    border: 1px solid #000;
+  }
+  .table {
+    border-collapse: collapse !important;
+  }
+  .table-bordered th,
+  .table-bordered td {
+    border: 1px solid #ddd !important;
+  }
+}
+* {
+  -webkit-box-sizing: border-box;
+     -moz-box-sizing: border-box;
+          box-sizing: border-box;
+}
+*:before,
+*:after {
+  -webkit-box-sizing: border-box;
+     -moz-box-sizing: border-box;
+          box-sizing: border-box;
+}
+html {
+  font-size: 62.5%;
+
+  -webkit-tap-highlight-color: rgba(0, 0, 0, 0);
+}
+body {
+  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #333;
+  background-color: #fff;
+}
+input,
+button,
+select,
+textarea {
+  font-family: inherit;
+  font-size: inherit;
+  line-height: inherit;
+}
+a {
+  color: #428bca;
+  text-decoration: none;
+}
+a:hover,
+a:focus {
+  color: #2a6496;
+  text-decoration: underline;
+}
+a:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+figure {
+  margin: 0;
+}
+img {
+  vertical-align: middle;
+}
+.img-responsive,
+.thumbnail > img,
+.thumbnail a > img,
+.carousel-inner > .item > img,
+.carousel-inner > .item > a > img {
+  display: block;
+  max-width: 100%;
+  height: auto;
+}
+.img-rounded {
+  border-radius: 6px;
+}
+.img-thumbnail {
+  display: inline-block;
+  max-width: 100%;
+  height: auto;
+  padding: 4px;
+  line-height: 1.42857143;
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-radius: 4px;
+  -webkit-transition: all .2s ease-in-out;
+          transition: all .2s ease-in-out;
+}
+.img-circle {
+  border-radius: 50%;
+}
+hr {
+  margin-top: 20px;
+  margin-bottom: 20px;
+  border: 0;
+  border-top: 1px solid #eee;
+}
+.sr-only {
+  position: absolute;
+  width: 1px;
+  height: 1px;
+  padding: 0;
+  margin: -1px;
+  overflow: hidden;
+  clip: rect(0, 0, 0, 0);
+  border: 0;
+}
+h1,
+h2,
+h3,
+h4,
+h5,
+h6,
+.h1,
+.h2,
+.h3,
+.h4,
+.h5,
+.h6 {
+  font-family: inherit;
+  font-weight: 500;
+  line-height: 1.1;
+  color: inherit;
+}
+h1 small,
+h2 small,
+h3 small,
+h4 small,
+h5 small,
+h6 small,
+.h1 small,
+.h2 small,
+.h3 small,
+.h4 small,
+.h5 small,
+.h6 small,
+h1 .small,
+h2 .small,
+h3 .small,
+h4 .small,
+h5 .small,
+h6 .small,
+.h1 .small,
+.h2 .small,
+.h3 .small,
+.h4 .small,
+.h5 .small,
+.h6 .small {
+  font-weight: normal;
+  line-height: 1;
+  color: #999;
+}
+h1,
+.h1,
+h2,
+.h2,
+h3,
+.h3 {
+  margin-top: 20px;
+  margin-bottom: 10px;
+}
+h1 small,
+.h1 small,
+h2 small,
+.h2 small,
+h3 small,
+.h3 small,
+h1 .small,
+.h1 .small,
+h2 .small,
+.h2 .small,
+h3 .small,
+.h3 .small {
+  font-size: 65%;
+}
+h4,
+.h4,
+h5,
+.h5,
+h6,
+.h6 {
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+h4 small,
+.h4 small,
+h5 small,
+.h5 small,
+h6 small,
+.h6 small,
+h4 .small,
+.h4 .small,
+h5 .small,
+.h5 .small,
+h6 .small,
+.h6 .small {
+  font-size: 75%;
+}
+h1,
+.h1 {
+  font-size: 36px;
+}
+h2,
+.h2 {
+  font-size: 30px;
+}
+h3,
+.h3 {
+  font-size: 24px;
+}
+h4,
+.h4 {
+  font-size: 18px;
+}
+h5,
+.h5 {
+  font-size: 14px;
+}
+h6,
+.h6 {
+  font-size: 12px;
+}
+p {
+  margin: 0 0 10px;
+}
+.lead {
+  margin-bottom: 20px;
+  font-size: 16px;
+  font-weight: 200;
+  line-height: 1.4;
+}
+@media (min-width: 768px) {
+  .lead {
+    font-size: 21px;
+  }
+}
+small,
+.small {
+  font-size: 85%;
+}
+cite {
+  font-style: normal;
+}
+.text-left {
+  text-align: left;
+}
+.text-right {
+  text-align: right;
+}
+.text-center {
+  text-align: center;
+}
+.text-justify {
+  text-align: justify;
+}
+.text-muted {
+  color: #999;
+}
+.text-primary {
+  color: #428bca;
+}
+a.text-primary:hover {
+  color: #3071a9;
+}
+.text-success {
+  color: #3c763d;
+}
+a.text-success:hover {
+  color: #2b542c;
+}
+.text-info {
+  color: #31708f;
+}
+a.text-info:hover {
+  color: #245269;
+}
+.text-warning {
+  color: #8a6d3b;
+}
+a.text-warning:hover {
+  color: #66512c;
+}
+.text-danger {
+  color: #a94442;
+}
+a.text-danger:hover {
+  color: #843534;
+}
+.bg-primary {
+  color: #fff;
+  background-color: #428bca;
+}
+a.bg-primary:hover {
+  background-color: #3071a9;
+}
+.bg-success {
+  background-color: #dff0d8;
+}
+a.bg-success:hover {
+  background-color: #c1e2b3;
+}
+.bg-info {
+  background-color: #d9edf7;
+}
+a.bg-info:hover {
+  background-color: #afd9ee;
+}
+.bg-warning {
+  background-color: #fcf8e3;
+}
+a.bg-warning:hover {
+  background-color: #f7ecb5;
+}
+.bg-danger {
+  background-color: #f2dede;
+}
+a.bg-danger:hover {
+  background-color: #e4b9b9;
+}
+.page-header {
+  padding-bottom: 9px;
+  margin: 40px 0 20px;
+  border-bottom: 1px solid #eee;
+}
+ul,
+ol {
+  margin-top: 0;
+  margin-bottom: 10px;
+}
+ul ul,
+ol ul,
+ul ol,
+ol ol {
+  margin-bottom: 0;
+}
+.list-unstyled {
+  padding-left: 0;
+  list-style: none;
+}
+.list-inline {
+  padding-left: 0;
+  margin-left: -5px;
+  list-style: none;
+}
+.list-inline > li {
+  display: inline-block;
+  padding-right: 5px;
+  padding-left: 5px;
+}
+dl {
+  margin-top: 0;
+  margin-bottom: 20px;
+}
+dt,
+dd {
+  line-height: 1.42857143;
+}
+dt {
+  font-weight: bold;
+}
+dd {
+  margin-left: 0;
+}
+@media (min-width: 768px) {
+  .dl-horizontal dt {
+    float: left;
+    width: 160px;
+    overflow: hidden;
+    clear: left;
+    text-align: right;
+    text-overflow: ellipsis;
+    white-space: nowrap;
+  }
+  .dl-horizontal dd {
+    margin-left: 180px;
+  }
+}
+abbr[title],
+abbr[data-original-title] {
+  cursor: help;
+  border-bottom: 1px dotted #999;
+}
+.initialism {
+  font-size: 90%;
+  text-transform: uppercase;
+}
+blockquote {
+  padding: 10px 20px;
+  margin: 0 0 20px;
+  font-size: 17.5px;
+  border-left: 5px solid #eee;
+}
+blockquote p:last-child,
+blockquote ul:last-child,
+blockquote ol:last-child {
+  margin-bottom: 0;
+}
+blockquote footer,
+blockquote small,
+blockquote .small {
+  display: block;
+  font-size: 80%;
+  line-height: 1.42857143;
+  color: #999;
+}
+blockquote footer:before,
+blockquote small:before,
+blockquote .small:before {
+  content: '\2014 \00A0';
+}
+.blockquote-reverse,
+blockquote.pull-right {
+  padding-right: 15px;
+  padding-left: 0;
+  text-align: right;
+  border-right: 5px solid #eee;
+  border-left: 0;
+}
+.blockquote-reverse footer:before,
+blockquote.pull-right footer:before,
+.blockquote-reverse small:before,
+blockquote.pull-right small:before,
+.blockquote-reverse .small:before,
+blockquote.pull-right .small:before {
+  content: '';
+}
+.blockquote-reverse footer:after,
+blockquote.pull-right footer:after,
+.blockquote-reverse small:after,
+blockquote.pull-right small:after,
+.blockquote-reverse .small:after,
+blockquote.pull-right .small:after {
+  content: '\00A0 \2014';
+}
+blockquote:before,
+blockquote:after {
+  content: "";
+}
+address {
+  margin-bottom: 20px;
+  font-style: normal;
+  line-height: 1.42857143;
+}
+code,
+kbd,
+pre,
+samp {
+  font-family: Menlo, Monaco, Consolas, "Courier New", monospace;
+}
+code {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #c7254e;
+  white-space: nowrap;
+  background-color: #f9f2f4;
+  border-radius: 4px;
+}
+kbd {
+  padding: 2px 4px;
+  font-size: 90%;
+  color: #fff;
+  background-color: #333;
+  border-radius: 3px;
+  box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .25);
+}
+pre {
+  display: block;
+  padding: 9.5px;
+  margin: 0 0 10px;
+  font-size: 13px;
+  line-height: 1.42857143;
+  color: #333;
+  word-break: break-all;
+  word-wrap: break-word;
+  background-color: #f5f5f5;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+}
+pre code {
+  padding: 0;
+  font-size: inherit;
+  color: inherit;
+  white-space: pre-wrap;
+  background-color: transparent;
+  border-radius: 0;
+}
+.pre-scrollable {
+  max-height: 340px;
+  overflow-y: scroll;
+}
+.container {
+  padding-right: 15px;
+  padding-left: 15px;
+  margin-right: auto;
+  margin-left: auto;
+}
+@media (min-width: 768px) {
+  .container {
+    width: 750px;
+  }
+}
+@media (min-width: 992px) {
+  .container {
+    width: 970px;
+  }
+}
+@media (min-width: 1200px) {
+  .container {
+    width: 1170px;
+  }
+}
+.container-fluid {
+  padding-right: 15px;
+  padding-left: 15px;
+  margin-right: auto;
+  margin-left: auto;
+}
+.row {
+  margin-right: -15px;
+  margin-left: -15px;
+}
+.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {
+  position: relative;
+  min-height: 1px;
+  padding-right: 15px;
+  padding-left: 15px;
+}
+.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {
+  float: left;
+}
+.col-xs-12 {
+  width: 100%;
+}
+.col-xs-11 {
+  width: 91.66666667%;
+}
+.col-xs-10 {
+  width: 83.33333333%;
+}
+.col-xs-9 {
+  width: 75%;
+}
+.col-xs-8 {
+  width: 66.66666667%;
+}
+.col-xs-7 {
+  width: 58.33333333%;
+}
+.col-xs-6 {
+  width: 50%;
+}
+.col-xs-5 {
+  width: 41.66666667%;
+}
+.col-xs-4 {
+  width: 33.33333333%;
+}
+.col-xs-3 {
+  width: 25%;
+}
+.col-xs-2 {
+  width: 16.66666667%;
+}
+.col-xs-1 {
+  width: 8.33333333%;
+}
+.col-xs-pull-12 {
+  right: 100%;
+}
+.col-xs-pull-11 {
+  right: 91.66666667%;
+}
+.col-xs-pull-10 {
+  right: 83.33333333%;
+}
+.col-xs-pull-9 {
+  right: 75%;
+}
+.col-xs-pull-8 {
+  right: 66.66666667%;
+}
+.col-xs-pull-7 {
+  right: 58.33333333%;
+}
+.col-xs-pull-6 {
+  right: 50%;
+}
+.col-xs-pull-5 {
+  right: 41.66666667%;
+}
+.col-xs-pull-4 {
+  right: 33.33333333%;
+}
+.col-xs-pull-3 {
+  right: 25%;
+}
+.col-xs-pull-2 {
+  right: 16.66666667%;
+}
+.col-xs-pull-1 {
+  right: 8.33333333%;
+}
+.col-xs-pull-0 {
+  right: 0;
+}
+.col-xs-push-12 {
+  left: 100%;
+}
+.col-xs-push-11 {
+  left: 91.66666667%;
+}
+.col-xs-push-10 {
+  left: 83.33333333%;
+}
+.col-xs-push-9 {
+  left: 75%;
+}
+.col-xs-push-8 {
+  left: 66.66666667%;
+}
+.col-xs-push-7 {
+  left: 58.33333333%;
+}
+.col-xs-push-6 {
+  left: 50%;
+}
+.col-xs-push-5 {
+  left: 41.66666667%;
+}
+.col-xs-push-4 {
+  left: 33.33333333%;
+}
+.col-xs-push-3 {
+  left: 25%;
+}
+.col-xs-push-2 {
+  left: 16.66666667%;
+}
+.col-xs-push-1 {
+  left: 8.33333333%;
+}
+.col-xs-push-0 {
+  left: 0;
+}
+.col-xs-offset-12 {
+  margin-left: 100%;
+}
+.col-xs-offset-11 {
+  margin-left: 91.66666667%;
+}
+.col-xs-offset-10 {
+  margin-left: 83.33333333%;
+}
+.col-xs-offset-9 {
+  margin-left: 75%;
+}
+.col-xs-offset-8 {
+  margin-left: 66.66666667%;
+}
+.col-xs-offset-7 {
+  margin-left: 58.33333333%;
+}
+.col-xs-offset-6 {
+  margin-left: 50%;
+}
+.col-xs-offset-5 {
+  margin-left: 41.66666667%;
+}
+.col-xs-offset-4 {
+  margin-left: 33.33333333%;
+}
+.col-xs-offset-3 {
+  margin-left: 25%;
+}
+.col-xs-offset-2 {
+  margin-left: 16.66666667%;
+}
+.col-xs-offset-1 {
+  margin-left: 8.33333333%;
+}
+.col-xs-offset-0 {
+  margin-left: 0;
+}
+@media (min-width: 768px) {
+  .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {
+    float: left;
+  }
+  .col-sm-12 {
+    width: 100%;
+  }
+  .col-sm-11 {
+    width: 91.66666667%;
+  }
+  .col-sm-10 {
+    width: 83.33333333%;
+  }
+  .col-sm-9 {
+    width: 75%;
+  }
+  .col-sm-8 {
+    width: 66.66666667%;
+  }
+  .col-sm-7 {
+    width: 58.33333333%;
+  }
+  .col-sm-6 {
+    width: 50%;
+  }
+  .col-sm-5 {
+    width: 41.66666667%;
+  }
+  .col-sm-4 {
+    width: 33.33333333%;
+  }
+  .col-sm-3 {
+    width: 25%;
+  }
+  .col-sm-2 {
+    width: 16.66666667%;
+  }
+  .col-sm-1 {
+    width: 8.33333333%;
+  }
+  .col-sm-pull-12 {
+    right: 100%;
+  }
+  .col-sm-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-sm-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-sm-pull-9 {
+    right: 75%;
+  }
+  .col-sm-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-sm-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-sm-pull-6 {
+    right: 50%;
+  }
+  .col-sm-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-sm-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-sm-pull-3 {
+    right: 25%;
+  }
+  .col-sm-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-sm-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-sm-pull-0 {
+    right: 0;
+  }
+  .col-sm-push-12 {
+    left: 100%;
+  }
+  .col-sm-push-11 {
+    left: 91.66666667%;
+  }
+  .col-sm-push-10 {
+    left: 83.33333333%;
+  }
+  .col-sm-push-9 {
+    left: 75%;
+  }
+  .col-sm-push-8 {
+    left: 66.66666667%;
+  }
+  .col-sm-push-7 {
+    left: 58.33333333%;
+  }
+  .col-sm-push-6 {
+    left: 50%;
+  }
+  .col-sm-push-5 {
+    left: 41.66666667%;
+  }
+  .col-sm-push-4 {
+    left: 33.33333333%;
+  }
+  .col-sm-push-3 {
+    left: 25%;
+  }
+  .col-sm-push-2 {
+    left: 16.66666667%;
+  }
+  .col-sm-push-1 {
+    left: 8.33333333%;
+  }
+  .col-sm-push-0 {
+    left: 0;
+  }
+  .col-sm-offset-12 {
+    margin-left: 100%;
+  }
+  .col-sm-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-sm-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-sm-offset-9 {
+    margin-left: 75%;
+  }
+  .col-sm-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-sm-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-sm-offset-6 {
+    margin-left: 50%;
+  }
+  .col-sm-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-sm-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-sm-offset-3 {
+    margin-left: 25%;
+  }
+  .col-sm-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-sm-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-sm-offset-0 {
+    margin-left: 0;
+  }
+}
+@media (min-width: 992px) {
+  .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {
+    float: left;
+  }
+  .col-md-12 {
+    width: 100%;
+  }
+  .col-md-11 {
+    width: 91.66666667%;
+  }
+  .col-md-10 {
+    width: 83.33333333%;
+  }
+  .col-md-9 {
+    width: 75%;
+  }
+  .col-md-8 {
+    width: 66.66666667%;
+  }
+  .col-md-7 {
+    width: 58.33333333%;
+  }
+  .col-md-6 {
+    width: 50%;
+  }
+  .col-md-5 {
+    width: 41.66666667%;
+  }
+  .col-md-4 {
+    width: 33.33333333%;
+  }
+  .col-md-3 {
+    width: 25%;
+  }
+  .col-md-2 {
+    width: 16.66666667%;
+  }
+  .col-md-1 {
+    width: 8.33333333%;
+  }
+  .col-md-pull-12 {
+    right: 100%;
+  }
+  .col-md-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-md-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-md-pull-9 {
+    right: 75%;
+  }
+  .col-md-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-md-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-md-pull-6 {
+    right: 50%;
+  }
+  .col-md-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-md-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-md-pull-3 {
+    right: 25%;
+  }
+  .col-md-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-md-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-md-pull-0 {
+    right: 0;
+  }
+  .col-md-push-12 {
+    left: 100%;
+  }
+  .col-md-push-11 {
+    left: 91.66666667%;
+  }
+  .col-md-push-10 {
+    left: 83.33333333%;
+  }
+  .col-md-push-9 {
+    left: 75%;
+  }
+  .col-md-push-8 {
+    left: 66.66666667%;
+  }
+  .col-md-push-7 {
+    left: 58.33333333%;
+  }
+  .col-md-push-6 {
+    left: 50%;
+  }
+  .col-md-push-5 {
+    left: 41.66666667%;
+  }
+  .col-md-push-4 {
+    left: 33.33333333%;
+  }
+  .col-md-push-3 {
+    left: 25%;
+  }
+  .col-md-push-2 {
+    left: 16.66666667%;
+  }
+  .col-md-push-1 {
+    left: 8.33333333%;
+  }
+  .col-md-push-0 {
+    left: 0;
+  }
+  .col-md-offset-12 {
+    margin-left: 100%;
+  }
+  .col-md-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-md-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-md-offset-9 {
+    margin-left: 75%;
+  }
+  .col-md-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-md-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-md-offset-6 {
+    margin-left: 50%;
+  }
+  .col-md-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-md-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-md-offset-3 {
+    margin-left: 25%;
+  }
+  .col-md-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-md-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-md-offset-0 {
+    margin-left: 0;
+  }
+}
+@media (min-width: 1200px) {
+  .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
+    float: left;
+  }
+  .col-lg-12 {
+    width: 100%;
+  }
+  .col-lg-11 {
+    width: 91.66666667%;
+  }
+  .col-lg-10 {
+    width: 83.33333333%;
+  }
+  .col-lg-9 {
+    width: 75%;
+  }
+  .col-lg-8 {
+    width: 66.66666667%;
+  }
+  .col-lg-7 {
+    width: 58.33333333%;
+  }
+  .col-lg-6 {
+    width: 50%;
+  }
+  .col-lg-5 {
+    width: 41.66666667%;
+  }
+  .col-lg-4 {
+    width: 33.33333333%;
+  }
+  .col-lg-3 {
+    width: 25%;
+  }
+  .col-lg-2 {
+    width: 16.66666667%;
+  }
+  .col-lg-1 {
+    width: 8.33333333%;
+  }
+  .col-lg-pull-12 {
+    right: 100%;
+  }
+  .col-lg-pull-11 {
+    right: 91.66666667%;
+  }
+  .col-lg-pull-10 {
+    right: 83.33333333%;
+  }
+  .col-lg-pull-9 {
+    right: 75%;
+  }
+  .col-lg-pull-8 {
+    right: 66.66666667%;
+  }
+  .col-lg-pull-7 {
+    right: 58.33333333%;
+  }
+  .col-lg-pull-6 {
+    right: 50%;
+  }
+  .col-lg-pull-5 {
+    right: 41.66666667%;
+  }
+  .col-lg-pull-4 {
+    right: 33.33333333%;
+  }
+  .col-lg-pull-3 {
+    right: 25%;
+  }
+  .col-lg-pull-2 {
+    right: 16.66666667%;
+  }
+  .col-lg-pull-1 {
+    right: 8.33333333%;
+  }
+  .col-lg-pull-0 {
+    right: 0;
+  }
+  .col-lg-push-12 {
+    left: 100%;
+  }
+  .col-lg-push-11 {
+    left: 91.66666667%;
+  }
+  .col-lg-push-10 {
+    left: 83.33333333%;
+  }
+  .col-lg-push-9 {
+    left: 75%;
+  }
+  .col-lg-push-8 {
+    left: 66.66666667%;
+  }
+  .col-lg-push-7 {
+    left: 58.33333333%;
+  }
+  .col-lg-push-6 {
+    left: 50%;
+  }
+  .col-lg-push-5 {
+    left: 41.66666667%;
+  }
+  .col-lg-push-4 {
+    left: 33.33333333%;
+  }
+  .col-lg-push-3 {
+    left: 25%;
+  }
+  .col-lg-push-2 {
+    left: 16.66666667%;
+  }
+  .col-lg-push-1 {
+    left: 8.33333333%;
+  }
+  .col-lg-push-0 {
+    left: 0;
+  }
+  .col-lg-offset-12 {
+    margin-left: 100%;
+  }
+  .col-lg-offset-11 {
+    margin-left: 91.66666667%;
+  }
+  .col-lg-offset-10 {
+    margin-left: 83.33333333%;
+  }
+  .col-lg-offset-9 {
+    margin-left: 75%;
+  }
+  .col-lg-offset-8 {
+    margin-left: 66.66666667%;
+  }
+  .col-lg-offset-7 {
+    margin-left: 58.33333333%;
+  }
+  .col-lg-offset-6 {
+    margin-left: 50%;
+  }
+  .col-lg-offset-5 {
+    margin-left: 41.66666667%;
+  }
+  .col-lg-offset-4 {
+    margin-left: 33.33333333%;
+  }
+  .col-lg-offset-3 {
+    margin-left: 25%;
+  }
+  .col-lg-offset-2 {
+    margin-left: 16.66666667%;
+  }
+  .col-lg-offset-1 {
+    margin-left: 8.33333333%;
+  }
+  .col-lg-offset-0 {
+    margin-left: 0;
+  }
+}
+table {
+  max-width: 100%;
+  background-color: transparent;
+}
+th {
+  text-align: left;
+}
+.table {
+  width: 100%;
+  margin-bottom: 20px;
+}
+.table > thead > tr > th,
+.table > tbody > tr > th,
+.table > tfoot > tr > th,
+.table > thead > tr > td,
+.table > tbody > tr > td,
+.table > tfoot > tr > td {
+  padding: 8px;
+  line-height: 1.42857143;
+  vertical-align: top;
+  border-top: 1px solid #ddd;
+}
+.table > thead > tr > th {
+  vertical-align: bottom;
+  border-bottom: 2px solid #ddd;
+}
+.table > caption + thead > tr:first-child > th,
+.table > colgroup + thead > tr:first-child > th,
+.table > thead:first-child > tr:first-child > th,
+.table > caption + thead > tr:first-child > td,
+.table > colgroup + thead > tr:first-child > td,
+.table > thead:first-child > tr:first-child > td {
+  border-top: 0;
+}
+.table > tbody + tbody {
+  border-top: 2px solid #ddd;
+}
+.table .table {
+  background-color: #fff;
+}
+.table-condensed > thead > tr > th,
+.table-condensed > tbody > tr > th,
+.table-condensed > tfoot > tr > th,
+.table-condensed > thead > tr > td,
+.table-condensed > tbody > tr > td,
+.table-condensed > tfoot > tr > td {
+  padding: 5px;
+}
+.table-bordered {
+  border: 1px solid #ddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > tbody > tr > th,
+.table-bordered > tfoot > tr > th,
+.table-bordered > thead > tr > td,
+.table-bordered > tbody > tr > td,
+.table-bordered > tfoot > tr > td {
+  border: 1px solid #ddd;
+}
+.table-bordered > thead > tr > th,
+.table-bordered > thead > tr > td {
+  border-bottom-width: 2px;
+}
+.table-striped > tbody > tr:nth-child(odd) > td,
+.table-striped > tbody > tr:nth-child(odd) > th {
+  background-color: #f9f9f9;
+}
+.table-hover > tbody > tr:hover > td,
+.table-hover > tbody > tr:hover > th {
+  background-color: #f5f5f5;
+}
+table col[class*="col-"] {
+  position: static;
+  display: table-column;
+  float: none;
+}
+table td[class*="col-"],
+table th[class*="col-"] {
+  position: static;
+  display: table-cell;
+  float: none;
+}
+.table > thead > tr > td.active,
+.table > tbody > tr > td.active,
+.table > tfoot > tr > td.active,
+.table > thead > tr > th.active,
+.table > tbody > tr > th.active,
+.table > tfoot > tr > th.active,
+.table > thead > tr.active > td,
+.table > tbody > tr.active > td,
+.table > tfoot > tr.active > td,
+.table > thead > tr.active > th,
+.table > tbody > tr.active > th,
+.table > tfoot > tr.active > th {
+  background-color: #f5f5f5;
+}
+.table-hover > tbody > tr > td.active:hover,
+.table-hover > tbody > tr > th.active:hover,
+.table-hover > tbody > tr.active:hover > td,
+.table-hover > tbody > tr.active:hover > th {
+  background-color: #e8e8e8;
+}
+.table > thead > tr > td.success,
+.table > tbody > tr > td.success,
+.table > tfoot > tr > td.success,
+.table > thead > tr > th.success,
+.table > tbody > tr > th.success,
+.table > tfoot > tr > th.success,
+.table > thead > tr.success > td,
+.table > tbody > tr.success > td,
+.table > tfoot > tr.success > td,
+.table > thead > tr.success > th,
+.table > tbody > tr.success > th,
+.table > tfoot > tr.success > th {
+  background-color: #dff0d8;
+}
+.table-hover > tbody > tr > td.success:hover,
+.table-hover > tbody > tr > th.success:hover,
+.table-hover > tbody > tr.success:hover > td,
+.table-hover > tbody > tr.success:hover > th {
+  background-color: #d0e9c6;
+}
+.table > thead > tr > td.info,
+.table > tbody > tr > td.info,
+.table > tfoot > tr > td.info,
+.table > thead > tr > th.info,
+.table > tbody > tr > th.info,
+.table > tfoot > tr > th.info,
+.table > thead > tr.info > td,
+.table > tbody > tr.info > td,
+.table > tfoot > tr.info > td,
+.table > thead > tr.info > th,
+.table > tbody > tr.info > th,
+.table > tfoot > tr.info > th {
+  background-color: #d9edf7;
+}
+.table-hover > tbody > tr > td.info:hover,
+.table-hover > tbody > tr > th.info:hover,
+.table-hover > tbody > tr.info:hover > td,
+.table-hover > tbody > tr.info:hover > th {
+  background-color: #c4e3f3;
+}
+.table > thead > tr > td.warning,
+.table > tbody > tr > td.warning,
+.table > tfoot > tr > td.warning,
+.table > thead > tr > th.warning,
+.table > tbody > tr > th.warning,
+.table > tfoot > tr > th.warning,
+.table > thead > tr.warning > td,
+.table > tbody > tr.warning > td,
+.table > tfoot > tr.warning > td,
+.table > thead > tr.warning > th,
+.table > tbody > tr.warning > th,
+.table > tfoot > tr.warning > th {
+  background-color: #fcf8e3;
+}
+.table-hover > tbody > tr > td.warning:hover,
+.table-hover > tbody > tr > th.warning:hover,
+.table-hover > tbody > tr.warning:hover > td,
+.table-hover > tbody > tr.warning:hover > th {
+  background-color: #faf2cc;
+}
+.table > thead > tr > td.danger,
+.table > tbody > tr > td.danger,
+.table > tfoot > tr > td.danger,
+.table > thead > tr > th.danger,
+.table > tbody > tr > th.danger,
+.table > tfoot > tr > th.danger,
+.table > thead > tr.danger > td,
+.table > tbody > tr.danger > td,
+.table > tfoot > tr.danger > td,
+.table > thead > tr.danger > th,
+.table > tbody > tr.danger > th,
+.table > tfoot > tr.danger > th {
+  background-color: #f2dede;
+}
+.table-hover > tbody > tr > td.danger:hover,
+.table-hover > tbody > tr > th.danger:hover,
+.table-hover > tbody > tr.danger:hover > td,
+.table-hover > tbody > tr.danger:hover > th {
+  background-color: #ebcccc;
+}
+@media (max-width: 767px) {
+  .table-responsive {
+    width: 100%;
+    margin-bottom: 15px;
+    overflow-x: scroll;
+    overflow-y: hidden;
+    -webkit-overflow-scrolling: touch;
+    -ms-overflow-style: -ms-autohiding-scrollbar;
+    border: 1px solid #ddd;
+  }
+  .table-responsive > .table {
+    margin-bottom: 0;
+  }
+  .table-responsive > .table > thead > tr > th,
+  .table-responsive > .table > tbody > tr > th,
+  .table-responsive > .table > tfoot > tr > th,
+  .table-responsive > .table > thead > tr > td,
+  .table-responsive > .table > tbody > tr > td,
+  .table-responsive > .table > tfoot > tr > td {
+    white-space: nowrap;
+  }
+  .table-responsive > .table-bordered {
+    border: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:first-child,
+  .table-responsive > .table-bordered > tbody > tr > th:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:first-child,
+  .table-responsive > .table-bordered > thead > tr > td:first-child,
+  .table-responsive > .table-bordered > tbody > tr > td:first-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:first-child {
+    border-left: 0;
+  }
+  .table-responsive > .table-bordered > thead > tr > th:last-child,
+  .table-responsive > .table-bordered > tbody > tr > th:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > th:last-child,
+  .table-responsive > .table-bordered > thead > tr > td:last-child,
+  .table-responsive > .table-bordered > tbody > tr > td:last-child,
+  .table-responsive > .table-bordered > tfoot > tr > td:last-child {
+    border-right: 0;
+  }
+  .table-responsive > .table-bordered > tbody > tr:last-child > th,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > th,
+  .table-responsive > .table-bordered > tbody > tr:last-child > td,
+  .table-responsive > .table-bordered > tfoot > tr:last-child > td {
+    border-bottom: 0;
+  }
+}
+fieldset {
+  min-width: 0;
+  padding: 0;
+  margin: 0;
+  border: 0;
+}
+legend {
+  display: block;
+  width: 100%;
+  padding: 0;
+  margin-bottom: 20px;
+  font-size: 21px;
+  line-height: inherit;
+  color: #333;
+  border: 0;
+  border-bottom: 1px solid #e5e5e5;
+}
+label {
+  display: inline-block;
+  margin-bottom: 5px;
+  font-weight: bold;
+}
+input[type="search"] {
+  -webkit-box-sizing: border-box;
+     -moz-box-sizing: border-box;
+          box-sizing: border-box;
+}
+input[type="radio"],
+input[type="checkbox"] {
+  margin: 4px 0 0;
+  margin-top: 1px \9;
+  /* IE8-9 */
+  line-height: normal;
+}
+input[type="file"] {
+  display: block;
+}
+input[type="range"] {
+  display: block;
+  width: 100%;
+}
+select[multiple],
+select[size] {
+  height: auto;
+}
+input[type="file"]:focus,
+input[type="radio"]:focus,
+input[type="checkbox"]:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+output {
+  display: block;
+  padding-top: 7px;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #555;
+}
+.form-control {
+  display: block;
+  width: 100%;
+  height: 34px;
+  padding: 6px 12px;
+  font-size: 14px;
+  line-height: 1.42857143;
+  color: #555;
+  background-color: #fff;
+  background-image: none;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+  -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+          transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
+}
+.form-control:focus {
+  border-color: #66afe9;
+  outline: 0;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6);
+          box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6);
+}
+.form-control::-moz-placeholder {
+  color: #999;
+  opacity: 1;
+}
+.form-control:-ms-input-placeholder {
+  color: #999;
+}
+.form-control::-webkit-input-placeholder {
+  color: #999;
+}
+.form-control[disabled],
+.form-control[readonly],
+fieldset[disabled] .form-control {
+  cursor: not-allowed;
+  background-color: #eee;
+  opacity: 1;
+}
+textarea.form-control {
+  height: auto;
+}
+input[type="search"] {
+  -webkit-appearance: none;
+}
+input[type="date"] {
+  line-height: 34px;
+}
+.form-group {
+  margin-bottom: 15px;
+}
+.radio,
+.checkbox {
+  display: block;
+  min-height: 20px;
+  padding-left: 20px;
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+.radio label,
+.checkbox label {
+  display: inline;
+  font-weight: normal;
+  cursor: pointer;
+}
+.radio input[type="radio"],
+.radio-inline input[type="radio"],
+.checkbox input[type="checkbox"],
+.checkbox-inline input[type="checkbox"] {
+  float: left;
+  margin-left: -20px;
+}
+.radio + .radio,
+.checkbox + .checkbox {
+  margin-top: -5px;
+}
+.radio-inline,
+.checkbox-inline {
+  display: inline-block;
+  padding-left: 20px;
+  margin-bottom: 0;
+  font-weight: normal;
+  vertical-align: middle;
+  cursor: pointer;
+}
+.radio-inline + .radio-inline,
+.checkbox-inline + .checkbox-inline {
+  margin-top: 0;
+  margin-left: 10px;
+}
+input[type="radio"][disabled],
+input[type="checkbox"][disabled],
+.radio[disabled],
+.radio-inline[disabled],
+.checkbox[disabled],
+.checkbox-inline[disabled],
+fieldset[disabled] input[type="radio"],
+fieldset[disabled] input[type="checkbox"],
+fieldset[disabled] .radio,
+fieldset[disabled] .radio-inline,
+fieldset[disabled] .checkbox,
+fieldset[disabled] .checkbox-inline {
+  cursor: not-allowed;
+}
+.input-sm {
+  height: 30px;
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+select.input-sm {
+  height: 30px;
+  line-height: 30px;
+}
+textarea.input-sm,
+select[multiple].input-sm {
+  height: auto;
+}
+.input-lg {
+  height: 46px;
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.33;
+  border-radius: 6px;
+}
+select.input-lg {
+  height: 46px;
+  line-height: 46px;
+}
+textarea.input-lg,
+select[multiple].input-lg {
+  height: auto;
+}
+.has-feedback {
+  position: relative;
+}
+.has-feedback .form-control {
+  padding-right: 42.5px;
+}
+.has-feedback .form-control-feedback {
+  position: absolute;
+  top: 25px;
+  right: 0;
+  display: block;
+  width: 34px;
+  height: 34px;
+  line-height: 34px;
+  text-align: center;
+}
+.has-success .help-block,
+.has-success .control-label,
+.has-success .radio,
+.has-success .checkbox,
+.has-success .radio-inline,
+.has-success .checkbox-inline {
+  color: #3c763d;
+}
+.has-success .form-control {
+  border-color: #3c763d;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+}
+.has-success .form-control:focus {
+  border-color: #2b542c;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168;
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168;
+}
+.has-success .input-group-addon {
+  color: #3c763d;
+  background-color: #dff0d8;
+  border-color: #3c763d;
+}
+.has-success .form-control-feedback {
+  color: #3c763d;
+}
+.has-warning .help-block,
+.has-warning .control-label,
+.has-warning .radio,
+.has-warning .checkbox,
+.has-warning .radio-inline,
+.has-warning .checkbox-inline {
+  color: #8a6d3b;
+}
+.has-warning .form-control {
+  border-color: #8a6d3b;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+}
+.has-warning .form-control:focus {
+  border-color: #66512c;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b;
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b;
+}
+.has-warning .input-group-addon {
+  color: #8a6d3b;
+  background-color: #fcf8e3;
+  border-color: #8a6d3b;
+}
+.has-warning .form-control-feedback {
+  color: #8a6d3b;
+}
+.has-error .help-block,
+.has-error .control-label,
+.has-error .radio,
+.has-error .checkbox,
+.has-error .radio-inline,
+.has-error .checkbox-inline {
+  color: #a94442;
+}
+.has-error .form-control {
+  border-color: #a94442;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075);
+}
+.has-error .form-control:focus {
+  border-color: #843534;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483;
+          box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483;
+}
+.has-error .input-group-addon {
+  color: #a94442;
+  background-color: #f2dede;
+  border-color: #a94442;
+}
+.has-error .form-control-feedback {
+  color: #a94442;
+}
+.form-control-static {
+  margin-bottom: 0;
+}
+.help-block {
+  display: block;
+  margin-top: 5px;
+  margin-bottom: 10px;
+  color: #737373;
+}
+@media (min-width: 768px) {
+  .form-inline .form-group {
+    display: inline-block;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .form-control {
+    display: inline-block;
+    width: auto;
+    vertical-align: middle;
+  }
+  .form-inline .input-group > .form-control {
+    width: 100%;
+  }
+  .form-inline .control-label {
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio,
+  .form-inline .checkbox {
+    display: inline-block;
+    padding-left: 0;
+    margin-top: 0;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .form-inline .radio input[type="radio"],
+  .form-inline .checkbox input[type="checkbox"] {
+    float: none;
+    margin-left: 0;
+  }
+  .form-inline .has-feedback .form-control-feedback {
+    top: 0;
+  }
+}
+.form-horizontal .control-label,
+.form-horizontal .radio,
+.form-horizontal .checkbox,
+.form-horizontal .radio-inline,
+.form-horizontal .checkbox-inline {
+  padding-top: 7px;
+  margin-top: 0;
+  margin-bottom: 0;
+}
+.form-horizontal .radio,
+.form-horizontal .checkbox {
+  min-height: 27px;
+}
+.form-horizontal .form-group {
+  margin-right: -15px;
+  margin-left: -15px;
+}
+.form-horizontal .form-control-static {
+  padding-top: 7px;
+}
+@media (min-width: 768px) {
+  .form-horizontal .control-label {
+    text-align: right;
+  }
+}
+.form-horizontal .has-feedback .form-control-feedback {
+  top: 0;
+  right: 15px;
+}
+.btn {
+  display: inline-block;
+  padding: 6px 12px;
+  margin-bottom: 0;
+  font-size: 14px;
+  font-weight: normal;
+  line-height: 1.42857143;
+  text-align: center;
+  white-space: nowrap;
+  vertical-align: middle;
+  cursor: pointer;
+  -webkit-user-select: none;
+     -moz-user-select: none;
+      -ms-user-select: none;
+          user-select: none;
+  background-image: none;
+  border: 1px solid transparent;
+  border-radius: 4px;
+}
+.btn:focus,
+.btn:active:focus,
+.btn.active:focus {
+  outline: thin dotted;
+  outline: 5px auto -webkit-focus-ring-color;
+  outline-offset: -2px;
+}
+.btn:hover,
+.btn:focus {
+  color: #333;
+  text-decoration: none;
+}
+.btn:active,
+.btn.active {
+  background-image: none;
+  outline: 0;
+  -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125);
+          box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125);
+}
+.btn.disabled,
+.btn[disabled],
+fieldset[disabled] .btn {
+  pointer-events: none;
+  cursor: not-allowed;
+  filter: alpha(opacity=65);
+  -webkit-box-shadow: none;
+          box-shadow: none;
+  opacity: .65;
+}
+.btn-default {
+  color: #333;
+  background-color: #fff;
+  border-color: #ccc;
+}
+.btn-default:hover,
+.btn-default:focus,
+.btn-default:active,
+.btn-default.active,
+.open .dropdown-toggle.btn-default {
+  color: #333;
+  background-color: #ebebeb;
+  border-color: #adadad;
+}
+.btn-default:active,
+.btn-default.active,
+.open .dropdown-toggle.btn-default {
+  background-image: none;
+}
+.btn-default.disabled,
+.btn-default[disabled],
+fieldset[disabled] .btn-default,
+.btn-default.disabled:hover,
+.btn-default[disabled]:hover,
+fieldset[disabled] .btn-default:hover,
+.btn-default.disabled:focus,
+.btn-default[disabled]:focus,
+fieldset[disabled] .btn-default:focus,
+.btn-default.disabled:active,
+.btn-default[disabled]:active,
+fieldset[disabled] .btn-default:active,
+.btn-default.disabled.active,
+.btn-default[disabled].active,
+fieldset[disabled] .btn-default.active {
+  background-color: #fff;
+  border-color: #ccc;
+}
+.btn-default .badge {
+  color: #fff;
+  background-color: #333;
+}
+.btn-primary {
+  color: #fff;
+  background-color: #428bca;
+  border-color: #357ebd;
+}
+.btn-primary:hover,
+.btn-primary:focus,
+.btn-primary:active,
+.btn-primary.active,
+.open .dropdown-toggle.btn-primary {
+  color: #fff;
+  background-color: #3276b1;
+  border-color: #285e8e;
+}
+.btn-primary:active,
+.btn-primary.active,
+.open .dropdown-toggle.btn-primary {
+  background-image: none;
+}
+.btn-primary.disabled,
+.btn-primary[disabled],
+fieldset[disabled] .btn-primary,
+.btn-primary.disabled:hover,
+.btn-primary[disabled]:hover,
+fieldset[disabled] .btn-primary:hover,
+.btn-primary.disabled:focus,
+.btn-primary[disabled]:focus,
+fieldset[disabled] .btn-primary:focus,
+.btn-primary.disabled:active,
+.btn-primary[disabled]:active,
+fieldset[disabled] .btn-primary:active,
+.btn-primary.disabled.active,
+.btn-primary[disabled].active,
+fieldset[disabled] .btn-primary.active {
+  background-color: #428bca;
+  border-color: #357ebd;
+}
+.btn-primary .badge {
+  color: #428bca;
+  background-color: #fff;
+}
+.btn-success {
+  color: #fff;
+  background-color: #5cb85c;
+  border-color: #4cae4c;
+}
+.btn-success:hover,
+.btn-success:focus,
+.btn-success:active,
+.btn-success.active,
+.open .dropdown-toggle.btn-success {
+  color: #fff;
+  background-color: #47a447;
+  border-color: #398439;
+}
+.btn-success:active,
+.btn-success.active,
+.open .dropdown-toggle.btn-success {
+  background-image: none;
+}
+.btn-success.disabled,
+.btn-success[disabled],
+fieldset[disabled] .btn-success,
+.btn-success.disabled:hover,
+.btn-success[disabled]:hover,
+fieldset[disabled] .btn-success:hover,
+.btn-success.disabled:focus,
+.btn-success[disabled]:focus,
+fieldset[disabled] .btn-success:focus,
+.btn-success.disabled:active,
+.btn-success[disabled]:active,
+fieldset[disabled] .btn-success:active,
+.btn-success.disabled.active,
+.btn-success[disabled].active,
+fieldset[disabled] .btn-success.active {
+  background-color: #5cb85c;
+  border-color: #4cae4c;
+}
+.btn-success .badge {
+  color: #5cb85c;
+  background-color: #fff;
+}
+.btn-info {
+  color: #fff;
+  background-color: #5bc0de;
+  border-color: #46b8da;
+}
+.btn-info:hover,
+.btn-info:focus,
+.btn-info:active,
+.btn-info.active,
+.open .dropdown-toggle.btn-info {
+  color: #fff;
+  background-color: #39b3d7;
+  border-color: #269abc;
+}
+.btn-info:active,
+.btn-info.active,
+.open .dropdown-toggle.btn-info {
+  background-image: none;
+}
+.btn-info.disabled,
+.btn-info[disabled],
+fieldset[disabled] .btn-info,
+.btn-info.disabled:hover,
+.btn-info[disabled]:hover,
+fieldset[disabled] .btn-info:hover,
+.btn-info.disabled:focus,
+.btn-info[disabled]:focus,
+fieldset[disabled] .btn-info:focus,
+.btn-info.disabled:active,
+.btn-info[disabled]:active,
+fieldset[disabled] .btn-info:active,
+.btn-info.disabled.active,
+.btn-info[disabled].active,
+fieldset[disabled] .btn-info.active {
+  background-color: #5bc0de;
+  border-color: #46b8da;
+}
+.btn-info .badge {
+  color: #5bc0de;
+  background-color: #fff;
+}
+.btn-warning {
+  color: #fff;
+  background-color: #f0ad4e;
+  border-color: #eea236;
+}
+.btn-warning:hover,
+.btn-warning:focus,
+.btn-warning:active,
+.btn-warning.active,
+.open .dropdown-toggle.btn-warning {
+  color: #fff;
+  background-color: #ed9c28;
+  border-color: #d58512;
+}
+.btn-warning:active,
+.btn-warning.active,
+.open .dropdown-toggle.btn-warning {
+  background-image: none;
+}
+.btn-warning.disabled,
+.btn-warning[disabled],
+fieldset[disabled] .btn-warning,
+.btn-warning.disabled:hover,
+.btn-warning[disabled]:hover,
+fieldset[disabled] .btn-warning:hover,
+.btn-warning.disabled:focus,
+.btn-warning[disabled]:focus,
+fieldset[disabled] .btn-warning:focus,
+.btn-warning.disabled:active,
+.btn-warning[disabled]:active,
+fieldset[disabled] .btn-warning:active,
+.btn-warning.disabled.active,
+.btn-warning[disabled].active,
+fieldset[disabled] .btn-warning.active {
+  background-color: #f0ad4e;
+  border-color: #eea236;
+}
+.btn-warning .badge {
+  color: #f0ad4e;
+  background-color: #fff;
+}
+.btn-danger {
+  color: #fff;
+  background-color: #d9534f;
+  border-color: #d43f3a;
+}
+.btn-danger:hover,
+.btn-danger:focus,
+.btn-danger:active,
+.btn-danger.active,
+.open .dropdown-toggle.btn-danger {
+  color: #fff;
+  background-color: #d2322d;
+  border-color: #ac2925;
+}
+.btn-danger:active,
+.btn-danger.active,
+.open .dropdown-toggle.btn-danger {
+  background-image: none;
+}
+.btn-danger.disabled,
+.btn-danger[disabled],
+fieldset[disabled] .btn-danger,
+.btn-danger.disabled:hover,
+.btn-danger[disabled]:hover,
+fieldset[disabled] .btn-danger:hover,
+.btn-danger.disabled:focus,
+.btn-danger[disabled]:focus,
+fieldset[disabled] .btn-danger:focus,
+.btn-danger.disabled:active,
+.btn-danger[disabled]:active,
+fieldset[disabled] .btn-danger:active,
+.btn-danger.disabled.active,
+.btn-danger[disabled].active,
+fieldset[disabled] .btn-danger.active {
+  background-color: #d9534f;
+  border-color: #d43f3a;
+}
+.btn-danger .badge {
+  color: #d9534f;
+  background-color: #fff;
+}
+.btn-link {
+  font-weight: normal;
+  color: #428bca;
+  cursor: pointer;
+  border-radius: 0;
+}
+.btn-link,
+.btn-link:active,
+.btn-link[disabled],
+fieldset[disabled] .btn-link {
+  background-color: transparent;
+  -webkit-box-shadow: none;
+          box-shadow: none;
+}
+.btn-link,
+.btn-link:hover,
+.btn-link:focus,
+.btn-link:active {
+  border-color: transparent;
+}
+.btn-link:hover,
+.btn-link:focus {
+  color: #2a6496;
+  text-decoration: underline;
+  background-color: transparent;
+}
+.btn-link[disabled]:hover,
+fieldset[disabled] .btn-link:hover,
+.btn-link[disabled]:focus,
+fieldset[disabled] .btn-link:focus {
+  color: #999;
+  text-decoration: none;
+}
+.btn-lg,
+.btn-group-lg > .btn {
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.33;
+  border-radius: 6px;
+}
+.btn-sm,
+.btn-group-sm > .btn {
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+.btn-xs,
+.btn-group-xs > .btn {
+  padding: 1px 5px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+.btn-block {
+  display: block;
+  width: 100%;
+  padding-right: 0;
+  padding-left: 0;
+}
+.btn-block + .btn-block {
+  margin-top: 5px;
+}
+input[type="submit"].btn-block,
+input[type="reset"].btn-block,
+input[type="button"].btn-block {
+  width: 100%;
+}
+.fade {
+  opacity: 0;
+  -webkit-transition: opacity .15s linear;
+          transition: opacity .15s linear;
+}
+.fade.in {
+  opacity: 1;
+}
+.collapse {
+  display: none;
+}
+.collapse.in {
+  display: block;
+}
+.collapsing {
+  position: relative;
+  height: 0;
+  overflow: hidden;
+  -webkit-transition: height .35s ease;
+          transition: height .35s ease;
+}
+@font-face {
+  font-family: 'Glyphicons Halflings';
+
+  src: url('../fonts/glyphicons-halflings-regular.eot');
+  src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');
+}
+.glyphicon {
+  position: relative;
+  top: 1px;
+  display: inline-block;
+  font-family: 'Glyphicons Halflings';
+  font-style: normal;
+  font-weight: normal;
+  line-height: 1;
+
+  -webkit-font-smoothing: antialiased;
+  -moz-osx-font-smoothing: grayscale;
+}
+.glyphicon-asterisk:before {
+  content: "\2a";
+}
+.glyphicon-plus:before {
+  content: "\2b";
+}
+.glyphicon-euro:before {
+  content: "\20ac";
+}
+.glyphicon-minus:before {
+  content: "\2212";
+}
+.glyphicon-cloud:before {
+  content: "\2601";
+}
+.glyphicon-envelope:before {
+  content: "\2709";
+}
+.glyphicon-pencil:before {
+  content: "\270f";
+}
+.glyphicon-glass:before {
+  content: "\e001";
+}
+.glyphicon-music:before {
+  content: "\e002";
+}
+.glyphicon-search:before {
+  content: "\e003";
+}
+.glyphicon-heart:before {
+  content: "\e005";
+}
+.glyphicon-star:before {
+  content: "\e006";
+}
+.glyphicon-star-empty:before {
+  content: "\e007";
+}
+.glyphicon-user:before {
+  content: "\e008";
+}
+.glyphicon-film:before {
+  content: "\e009";
+}
+.glyphicon-th-large:before {
+  content: "\e010";
+}
+.glyphicon-th:before {
+  content: "\e011";
+}
+.glyphicon-th-list:before {
+  content: "\e012";
+}
+.glyphicon-ok:before {
+  content: "\e013";
+}
+.glyphicon-remove:before {
+  content: "\e014";
+}
+.glyphicon-zoom-in:before {
+  content: "\e015";
+}
+.glyphicon-zoom-out:before {
+  content: "\e016";
+}
+.glyphicon-off:before {
+  content: "\e017";
+}
+.glyphicon-signal:before {
+  content: "\e018";
+}
+.glyphicon-cog:before {
+  content: "\e019";
+}
+.glyphicon-trash:before {
+  content: "\e020";
+}
+.glyphicon-home:before {
+  content: "\e021";
+}
+.glyphicon-file:before {
+  content: "\e022";
+}
+.glyphicon-time:before {
+  content: "\e023";
+}
+.glyphicon-road:before {
+  content: "\e024";
+}
+.glyphicon-download-alt:before {
+  content: "\e025";
+}
+.glyphicon-download:before {
+  content: "\e026";
+}
+.glyphicon-upload:before {
+  content: "\e027";
+}
+.glyphicon-inbox:before {
+  content: "\e028";
+}
+.glyphicon-play-circle:before {
+  content: "\e029";
+}
+.glyphicon-repeat:before {
+  content: "\e030";
+}
+.glyphicon-refresh:before {
+  content: "\e031";
+}
+.glyphicon-list-alt:before {
+  content: "\e032";
+}
+.glyphicon-lock:before {
+  content: "\e033";
+}
+.glyphicon-flag:before {
+  content: "\e034";
+}
+.glyphicon-headphones:before {
+  content: "\e035";
+}
+.glyphicon-volume-off:before {
+  content: "\e036";
+}
+.glyphicon-volume-down:before {
+  content: "\e037";
+}
+.glyphicon-volume-up:before {
+  content: "\e038";
+}
+.glyphicon-qrcode:before {
+  content: "\e039";
+}
+.glyphicon-barcode:before {
+  content: "\e040";
+}
+.glyphicon-tag:before {
+  content: "\e041";
+}
+.glyphicon-tags:before {
+  content: "\e042";
+}
+.glyphicon-book:before {
+  content: "\e043";
+}
+.glyphicon-bookmark:before {
+  content: "\e044";
+}
+.glyphicon-print:before {
+  content: "\e045";
+}
+.glyphicon-camera:before {
+  content: "\e046";
+}
+.glyphicon-font:before {
+  content: "\e047";
+}
+.glyphicon-bold:before {
+  content: "\e048";
+}
+.glyphicon-italic:before {
+  content: "\e049";
+}
+.glyphicon-text-height:before {
+  content: "\e050";
+}
+.glyphicon-text-width:before {
+  content: "\e051";
+}
+.glyphicon-align-left:before {
+  content: "\e052";
+}
+.glyphicon-align-center:before {
+  content: "\e053";
+}
+.glyphicon-align-right:before {
+  content: "\e054";
+}
+.glyphicon-align-justify:before {
+  content: "\e055";
+}
+.glyphicon-list:before {
+  content: "\e056";
+}
+.glyphicon-indent-left:before {
+  content: "\e057";
+}
+.glyphicon-indent-right:before {
+  content: "\e058";
+}
+.glyphicon-facetime-video:before {
+  content: "\e059";
+}
+.glyphicon-picture:before {
+  content: "\e060";
+}
+.glyphicon-map-marker:before {
+  content: "\e062";
+}
+.glyphicon-adjust:before {
+  content: "\e063";
+}
+.glyphicon-tint:before {
+  content: "\e064";
+}
+.glyphicon-edit:before {
+  content: "\e065";
+}
+.glyphicon-share:before {
+  content: "\e066";
+}
+.glyphicon-check:before {
+  content: "\e067";
+}
+.glyphicon-move:before {
+  content: "\e068";
+}
+.glyphicon-step-backward:before {
+  content: "\e069";
+}
+.glyphicon-fast-backward:before {
+  content: "\e070";
+}
+.glyphicon-backward:before {
+  content: "\e071";
+}
+.glyphicon-play:before {
+  content: "\e072";
+}
+.glyphicon-pause:before {
+  content: "\e073";
+}
+.glyphicon-stop:before {
+  content: "\e074";
+}
+.glyphicon-forward:before {
+  content: "\e075";
+}
+.glyphicon-fast-forward:before {
+  content: "\e076";
+}
+.glyphicon-step-forward:before {
+  content: "\e077";
+}
+.glyphicon-eject:before {
+  content: "\e078";
+}
+.glyphicon-chevron-left:before {
+  content: "\e079";
+}
+.glyphicon-chevron-right:before {
+  content: "\e080";
+}
+.glyphicon-plus-sign:before {
+  content: "\e081";
+}
+.glyphicon-minus-sign:before {
+  content: "\e082";
+}
+.glyphicon-remove-sign:before {
+  content: "\e083";
+}
+.glyphicon-ok-sign:before {
+  content: "\e084";
+}
+.glyphicon-question-sign:before {
+  content: "\e085";
+}
+.glyphicon-info-sign:before {
+  content: "\e086";
+}
+.glyphicon-screenshot:before {
+  content: "\e087";
+}
+.glyphicon-remove-circle:before {
+  content: "\e088";
+}
+.glyphicon-ok-circle:before {
+  content: "\e089";
+}
+.glyphicon-ban-circle:before {
+  content: "\e090";
+}
+.glyphicon-arrow-left:before {
+  content: "\e091";
+}
+.glyphicon-arrow-right:before {
+  content: "\e092";
+}
+.glyphicon-arrow-up:before {
+  content: "\e093";
+}
+.glyphicon-arrow-down:before {
+  content: "\e094";
+}
+.glyphicon-share-alt:before {
+  content: "\e095";
+}
+.glyphicon-resize-full:before {
+  content: "\e096";
+}
+.glyphicon-resize-small:before {
+  content: "\e097";
+}
+.glyphicon-exclamation-sign:before {
+  content: "\e101";
+}
+.glyphicon-gift:before {
+  content: "\e102";
+}
+.glyphicon-leaf:before {
+  content: "\e103";
+}
+.glyphicon-fire:before {
+  content: "\e104";
+}
+.glyphicon-eye-open:before {
+  content: "\e105";
+}
+.glyphicon-eye-close:before {
+  content: "\e106";
+}
+.glyphicon-warning-sign:before {
+  content: "\e107";
+}
+.glyphicon-plane:before {
+  content: "\e108";
+}
+.glyphicon-calendar:before {
+  content: "\e109";
+}
+.glyphicon-random:before {
+  content: "\e110";
+}
+.glyphicon-comment:before {
+  content: "\e111";
+}
+.glyphicon-magnet:before {
+  content: "\e112";
+}
+.glyphicon-chevron-up:before {
+  content: "\e113";
+}
+.glyphicon-chevron-down:before {
+  content: "\e114";
+}
+.glyphicon-retweet:before {
+  content: "\e115";
+}
+.glyphicon-shopping-cart:before {
+  content: "\e116";
+}
+.glyphicon-folder-close:before {
+  content: "\e117";
+}
+.glyphicon-folder-open:before {
+  content: "\e118";
+}
+.glyphicon-resize-vertical:before {
+  content: "\e119";
+}
+.glyphicon-resize-horizontal:before {
+  content: "\e120";
+}
+.glyphicon-hdd:before {
+  content: "\e121";
+}
+.glyphicon-bullhorn:before {
+  content: "\e122";
+}
+.glyphicon-bell:before {
+  content: "\e123";
+}
+.glyphicon-certificate:before {
+  content: "\e124";
+}
+.glyphicon-thumbs-up:before {
+  content: "\e125";
+}
+.glyphicon-thumbs-down:before {
+  content: "\e126";
+}
+.glyphicon-hand-right:before {
+  content: "\e127";
+}
+.glyphicon-hand-left:before {
+  content: "\e128";
+}
+.glyphicon-hand-up:before {
+  content: "\e129";
+}
+.glyphicon-hand-down:before {
+  content: "\e130";
+}
+.glyphicon-circle-arrow-right:before {
+  content: "\e131";
+}
+.glyphicon-circle-arrow-left:before {
+  content: "\e132";
+}
+.glyphicon-circle-arrow-up:before {
+  content: "\e133";
+}
+.glyphicon-circle-arrow-down:before {
+  content: "\e134";
+}
+.glyphicon-globe:before {
+  content: "\e135";
+}
+.glyphicon-wrench:before {
+  content: "\e136";
+}
+.glyphicon-tasks:before {
+  content: "\e137";
+}
+.glyphicon-filter:before {
+  content: "\e138";
+}
+.glyphicon-briefcase:before {
+  content: "\e139";
+}
+.glyphicon-fullscreen:before {
+  content: "\e140";
+}
+.glyphicon-dashboard:before {
+  content: "\e141";
+}
+.glyphicon-paperclip:before {
+  content: "\e142";
+}
+.glyphicon-heart-empty:before {
+  content: "\e143";
+}
+.glyphicon-link:before {
+  content: "\e144";
+}
+.glyphicon-phone:before {
+  content: "\e145";
+}
+.glyphicon-pushpin:before {
+  content: "\e146";
+}
+.glyphicon-usd:before {
+  content: "\e148";
+}
+.glyphicon-gbp:before {
+  content: "\e149";
+}
+.glyphicon-sort:before {
+  content: "\e150";
+}
+.glyphicon-sort-by-alphabet:before {
+  content: "\e151";
+}
+.glyphicon-sort-by-alphabet-alt:before {
+  content: "\e152";
+}
+.glyphicon-sort-by-order:before {
+  content: "\e153";
+}
+.glyphicon-sort-by-order-alt:before {
+  content: "\e154";
+}
+.glyphicon-sort-by-attributes:before {
+  content: "\e155";
+}
+.glyphicon-sort-by-attributes-alt:before {
+  content: "\e156";
+}
+.glyphicon-unchecked:before {
+  content: "\e157";
+}
+.glyphicon-expand:before {
+  content: "\e158";
+}
+.glyphicon-collapse-down:before {
+  content: "\e159";
+}
+.glyphicon-collapse-up:before {
+  content: "\e160";
+}
+.glyphicon-log-in:before {
+  content: "\e161";
+}
+.glyphicon-flash:before {
+  content: "\e162";
+}
+.glyphicon-log-out:before {
+  content: "\e163";
+}
+.glyphicon-new-window:before {
+  content: "\e164";
+}
+.glyphicon-record:before {
+  content: "\e165";
+}
+.glyphicon-save:before {
+  content: "\e166";
+}
+.glyphicon-open:before {
+  content: "\e167";
+}
+.glyphicon-saved:before {
+  content: "\e168";
+}
+.glyphicon-import:before {
+  content: "\e169";
+}
+.glyphicon-export:before {
+  content: "\e170";
+}
+.glyphicon-send:before {
+  content: "\e171";
+}
+.glyphicon-floppy-disk:before {
+  content: "\e172";
+}
+.glyphicon-floppy-saved:before {
+  content: "\e173";
+}
+.glyphicon-floppy-remove:before {
+  content: "\e174";
+}
+.glyphicon-floppy-save:before {
+  content: "\e175";
+}
+.glyphicon-floppy-open:before {
+  content: "\e176";
+}
+.glyphicon-credit-card:before {
+  content: "\e177";
+}
+.glyphicon-transfer:before {
+  content: "\e178";
+}
+.glyphicon-cutlery:before {
+  content: "\e179";
+}
+.glyphicon-header:before {
+  content: "\e180";
+}
+.glyphicon-compressed:before {
+  content: "\e181";
+}
+.glyphicon-earphone:before {
+  content: "\e182";
+}
+.glyphicon-phone-alt:before {
+  content: "\e183";
+}
+.glyphicon-tower:before {
+  content: "\e184";
+}
+.glyphicon-stats:before {
+  content: "\e185";
+}
+.glyphicon-sd-video:before {
+  content: "\e186";
+}
+.glyphicon-hd-video:before {
+  content: "\e187";
+}
+.glyphicon-subtitles:before {
+  content: "\e188";
+}
+.glyphicon-sound-stereo:before {
+  content: "\e189";
+}
+.glyphicon-sound-dolby:before {
+  content: "\e190";
+}
+.glyphicon-sound-5-1:before {
+  content: "\e191";
+}
+.glyphicon-sound-6-1:before {
+  content: "\e192";
+}
+.glyphicon-sound-7-1:before {
+  content: "\e193";
+}
+.glyphicon-copyright-mark:before {
+  content: "\e194";
+}
+.glyphicon-registration-mark:before {
+  content: "\e195";
+}
+.glyphicon-cloud-download:before {
+  content: "\e197";
+}
+.glyphicon-cloud-upload:before {
+  content: "\e198";
+}
+.glyphicon-tree-conifer:before {
+  content: "\e199";
+}
+.glyphicon-tree-deciduous:before {
+  content: "\e200";
+}
+.caret {
+  display: inline-block;
+  width: 0;
+  height: 0;
+  margin-left: 2px;
+  vertical-align: middle;
+  border-top: 4px solid;
+  border-right: 4px solid transparent;
+  border-left: 4px solid transparent;
+}
+.dropdown {
+  position: relative;
+}
+.dropdown-toggle:focus {
+  outline: 0;
+}
+.dropdown-menu {
+  position: absolute;
+  top: 100%;
+  left: 0;
+  z-index: 1000;
+  display: none;
+  float: left;
+  min-width: 160px;
+  padding: 5px 0;
+  margin: 2px 0 0;
+  font-size: 14px;
+  list-style: none;
+  background-color: #fff;
+  background-clip: padding-box;
+  border: 1px solid #ccc;
+  border: 1px solid rgba(0, 0, 0, .15);
+  border-radius: 4px;
+  -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, .175);
+          box-shadow: 0 6px 12px rgba(0, 0, 0, .175);
+}
+.dropdown-menu.pull-right {
+  right: 0;
+  left: auto;
+}
+.dropdown-menu .divider {
+  height: 1px;
+  margin: 9px 0;
+  overflow: hidden;
+  background-color: #e5e5e5;
+}
+.dropdown-menu > li > a {
+  display: block;
+  padding: 3px 20px;
+  clear: both;
+  font-weight: normal;
+  line-height: 1.42857143;
+  color: #333;
+  white-space: nowrap;
+}
+.dropdown-menu > li > a:hover,
+.dropdown-menu > li > a:focus {
+  color: #262626;
+  text-decoration: none;
+  background-color: #f5f5f5;
+}
+.dropdown-menu > .active > a,
+.dropdown-menu > .active > a:hover,
+.dropdown-menu > .active > a:focus {
+  color: #fff;
+  text-decoration: none;
+  background-color: #428bca;
+  outline: 0;
+}
+.dropdown-menu > .disabled > a,
+.dropdown-menu > .disabled > a:hover,
+.dropdown-menu > .disabled > a:focus {
+  color: #999;
+}
+.dropdown-menu > .disabled > a:hover,
+.dropdown-menu > .disabled > a:focus {
+  text-decoration: none;
+  cursor: not-allowed;
+  background-color: transparent;
+  background-image: none;
+  filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);
+}
+.open > .dropdown-menu {
+  display: block;
+}
+.open > a {
+  outline: 0;
+}
+.dropdown-menu-right {
+  right: 0;
+  left: auto;
+}
+.dropdown-menu-left {
+  right: auto;
+  left: 0;
+}
+.dropdown-header {
+  display: block;
+  padding: 3px 20px;
+  font-size: 12px;
+  line-height: 1.42857143;
+  color: #999;
+}
+.dropdown-backdrop {
+  position: fixed;
+  top: 0;
+  right: 0;
+  bottom: 0;
+  left: 0;
+  z-index: 990;
+}
+.pull-right > .dropdown-menu {
+  right: 0;
+  left: auto;
+}
+.dropup .caret,
+.navbar-fixed-bottom .dropdown .caret {
+  content: "";
+  border-top: 0;
+  border-bottom: 4px solid;
+}
+.dropup .dropdown-menu,
+.navbar-fixed-bottom .dropdown .dropdown-menu {
+  top: auto;
+  bottom: 100%;
+  margin-bottom: 1px;
+}
+@media (min-width: 768px) {
+  .navbar-right .dropdown-menu {
+    right: 0;
+    left: auto;
+  }
+  .navbar-right .dropdown-menu-left {
+    right: auto;
+    left: 0;
+  }
+}
+.btn-group,
+.btn-group-vertical {
+  position: relative;
+  display: inline-block;
+  vertical-align: middle;
+}
+.btn-group > .btn,
+.btn-group-vertical > .btn {
+  position: relative;
+  float: left;
+}
+.btn-group > .btn:hover,
+.btn-group-vertical > .btn:hover,
+.btn-group > .btn:focus,
+.btn-group-vertical > .btn:focus,
+.btn-group > .btn:active,
+.btn-group-vertical > .btn:active,
+.btn-group > .btn.active,
+.btn-group-vertical > .btn.active {
+  z-index: 2;
+}
+.btn-group > .btn:focus,
+.btn-group-vertical > .btn:focus {
+  outline: none;
+}
+.btn-group .btn + .btn,
+.btn-group .btn + .btn-group,
+.btn-group .btn-group + .btn,
+.btn-group .btn-group + .btn-group {
+  margin-left: -1px;
+}
+.btn-toolbar {
+  margin-left: -5px;
+}
+.btn-toolbar .btn-group,
+.btn-toolbar .input-group {
+  float: left;
+}
+.btn-toolbar > .btn,
+.btn-toolbar > .btn-group,
+.btn-toolbar > .input-group {
+  margin-left: 5px;
+}
+.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) {
+  border-radius: 0;
+}
+.btn-group > .btn:first-child {
+  margin-left: 0;
+}
+.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) {
+  border-top-right-radius: 0;
+  border-bottom-right-radius: 0;
+}
+.btn-group > .btn:last-child:not(:first-child),
+.btn-group > .dropdown-toggle:not(:first-child) {
+  border-top-left-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.btn-group > .btn-group {
+  float: left;
+}
+.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn {
+  border-radius: 0;
+}
+.btn-group > .btn-group:first-child > .btn:last-child,
+.btn-group > .btn-group:first-child > .dropdown-toggle {
+  border-top-right-radius: 0;
+  border-bottom-right-radius: 0;
+}
+.btn-group > .btn-group:last-child > .btn:first-child {
+  border-top-left-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.btn-group .dropdown-toggle:active,
+.btn-group.open .dropdown-toggle {
+  outline: 0;
+}
+.btn-group > .btn + .dropdown-toggle {
+  padding-right: 8px;
+  padding-left: 8px;
+}
+.btn-group > .btn-lg + .dropdown-toggle {
+  padding-right: 12px;
+  padding-left: 12px;
+}
+.btn-group.open .dropdown-toggle {
+  -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125);
+          box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125);
+}
+.btn-group.open .dropdown-toggle.btn-link {
+  -webkit-box-shadow: none;
+          box-shadow: none;
+}
+.btn .caret {
+  margin-left: 0;
+}
+.btn-lg .caret {
+  border-width: 5px 5px 0;
+  border-bottom-width: 0;
+}
+.dropup .btn-lg .caret {
+  border-width: 0 5px 5px;
+}
+.btn-group-vertical > .btn,
+.btn-group-vertical > .btn-group,
+.btn-group-vertical > .btn-group > .btn {
+  display: block;
+  float: none;
+  width: 100%;
+  max-width: 100%;
+}
+.btn-group-vertical > .btn-group > .btn {
+  float: none;
+}
+.btn-group-vertical > .btn + .btn,
+.btn-group-vertical > .btn + .btn-group,
+.btn-group-vertical > .btn-group + .btn,
+.btn-group-vertical > .btn-group + .btn-group {
+  margin-top: -1px;
+  margin-left: 0;
+}
+.btn-group-vertical > .btn:not(:first-child):not(:last-child) {
+  border-radius: 0;
+}
+.btn-group-vertical > .btn:first-child:not(:last-child) {
+  border-top-right-radius: 4px;
+  border-bottom-right-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.btn-group-vertical > .btn:last-child:not(:first-child) {
+  border-top-left-radius: 0;
+  border-top-right-radius: 0;
+  border-bottom-left-radius: 4px;
+}
+.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn {
+  border-radius: 0;
+}
+.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child,
+.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle {
+  border-bottom-right-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child {
+  border-top-left-radius: 0;
+  border-top-right-radius: 0;
+}
+.btn-group-justified {
+  display: table;
+  width: 100%;
+  table-layout: fixed;
+  border-collapse: separate;
+}
+.btn-group-justified > .btn,
+.btn-group-justified > .btn-group {
+  display: table-cell;
+  float: none;
+  width: 1%;
+}
+.btn-group-justified > .btn-group .btn {
+  width: 100%;
+}
+[data-toggle="buttons"] > .btn > input[type="radio"],
+[data-toggle="buttons"] > .btn > input[type="checkbox"] {
+  display: none;
+}
+.input-group {
+  position: relative;
+  display: table;
+  border-collapse: separate;
+}
+.input-group[class*="col-"] {
+  float: none;
+  padding-right: 0;
+  padding-left: 0;
+}
+.input-group .form-control {
+  position: relative;
+  z-index: 2;
+  float: left;
+  width: 100%;
+  margin-bottom: 0;
+}
+.input-group-lg > .form-control,
+.input-group-lg > .input-group-addon,
+.input-group-lg > .input-group-btn > .btn {
+  height: 46px;
+  padding: 10px 16px;
+  font-size: 18px;
+  line-height: 1.33;
+  border-radius: 6px;
+}
+select.input-group-lg > .form-control,
+select.input-group-lg > .input-group-addon,
+select.input-group-lg > .input-group-btn > .btn {
+  height: 46px;
+  line-height: 46px;
+}
+textarea.input-group-lg > .form-control,
+textarea.input-group-lg > .input-group-addon,
+textarea.input-group-lg > .input-group-btn > .btn,
+select[multiple].input-group-lg > .form-control,
+select[multiple].input-group-lg > .input-group-addon,
+select[multiple].input-group-lg > .input-group-btn > .btn {
+  height: auto;
+}
+.input-group-sm > .form-control,
+.input-group-sm > .input-group-addon,
+.input-group-sm > .input-group-btn > .btn {
+  height: 30px;
+  padding: 5px 10px;
+  font-size: 12px;
+  line-height: 1.5;
+  border-radius: 3px;
+}
+select.input-group-sm > .form-control,
+select.input-group-sm > .input-group-addon,
+select.input-group-sm > .input-group-btn > .btn {
+  height: 30px;
+  line-height: 30px;
+}
+textarea.input-group-sm > .form-control,
+textarea.input-group-sm > .input-group-addon,
+textarea.input-group-sm > .input-group-btn > .btn,
+select[multiple].input-group-sm > .form-control,
+select[multiple].input-group-sm > .input-group-addon,
+select[multiple].input-group-sm > .input-group-btn > .btn {
+  height: auto;
+}
+.input-group-addon,
+.input-group-btn,
+.input-group .form-control {
+  display: table-cell;
+}
+.input-group-addon:not(:first-child):not(:last-child),
+.input-group-btn:not(:first-child):not(:last-child),
+.input-group .form-control:not(:first-child):not(:last-child) {
+  border-radius: 0;
+}
+.input-group-addon,
+.input-group-btn {
+  width: 1%;
+  white-space: nowrap;
+  vertical-align: middle;
+}
+.input-group-addon {
+  padding: 6px 12px;
+  font-size: 14px;
+  font-weight: normal;
+  line-height: 1;
+  color: #555;
+  text-align: center;
+  background-color: #eee;
+  border: 1px solid #ccc;
+  border-radius: 4px;
+}
+.input-group-addon.input-sm {
+  padding: 5px 10px;
+  font-size: 12px;
+  border-radius: 3px;
+}
+.input-group-addon.input-lg {
+  padding: 10px 16px;
+  font-size: 18px;
+  border-radius: 6px;
+}
+.input-group-addon input[type="radio"],
+.input-group-addon input[type="checkbox"] {
+  margin-top: 0;
+}
+.input-group .form-control:first-child,
+.input-group-addon:first-child,
+.input-group-btn:first-child > .btn,
+.input-group-btn:first-child > .btn-group > .btn,
+.input-group-btn:first-child > .dropdown-toggle,
+.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle),
+.input-group-btn:last-child > .btn-group:not(:last-child) > .btn {
+  border-top-right-radius: 0;
+  border-bottom-right-radius: 0;
+}
+.input-group-addon:first-child {
+  border-right: 0;
+}
+.input-group .form-control:last-child,
+.input-group-addon:last-child,
+.input-group-btn:last-child > .btn,
+.input-group-btn:last-child > .btn-group > .btn,
+.input-group-btn:last-child > .dropdown-toggle,
+.input-group-btn:first-child > .btn:not(:first-child),
+.input-group-btn:first-child > .btn-group:not(:first-child) > .btn {
+  border-top-left-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.input-group-addon:last-child {
+  border-left: 0;
+}
+.input-group-btn {
+  position: relative;
+  font-size: 0;
+  white-space: nowrap;
+}
+.input-group-btn > .btn {
+  position: relative;
+}
+.input-group-btn > .btn + .btn {
+  margin-left: -1px;
+}
+.input-group-btn > .btn:hover,
+.input-group-btn > .btn:focus,
+.input-group-btn > .btn:active {
+  z-index: 2;
+}
+.input-group-btn:first-child > .btn,
+.input-group-btn:first-child > .btn-group {
+  margin-right: -1px;
+}
+.input-group-btn:last-child > .btn,
+.input-group-btn:last-child > .btn-group {
+  margin-left: -1px;
+}
+.nav {
+  padding-left: 0;
+  margin-bottom: 0;
+  list-style: none;
+}
+.nav > li {
+  position: relative;
+  display: block;
+}
+.nav > li > a {
+  position: relative;
+  display: block;
+  padding: 10px 15px;
+}
+.nav > li > a:hover,
+.nav > li > a:focus {
+  text-decoration: none;
+  background-color: #eee;
+}
+.nav > li.disabled > a {
+  color: #999;
+}
+.nav > li.disabled > a:hover,
+.nav > li.disabled > a:focus {
+  color: #999;
+  text-decoration: none;
+  cursor: not-allowed;
+  background-color: transparent;
+}
+.nav .open > a,
+.nav .open > a:hover,
+.nav .open > a:focus {
+  background-color: #eee;
+  border-color: #428bca;
+}
+.nav .nav-divider {
+  height: 1px;
+  margin: 9px 0;
+  overflow: hidden;
+  background-color: #e5e5e5;
+}
+.nav > li > a > img {
+  max-width: none;
+}
+.nav-tabs {
+  border-bottom: 1px solid #ddd;
+}
+.nav-tabs > li {
+  float: left;
+  margin-bottom: -1px;
+}
+.nav-tabs > li > a {
+  margin-right: 2px;
+  line-height: 1.42857143;
+  border: 1px solid transparent;
+  border-radius: 4px 4px 0 0;
+}
+.nav-tabs > li > a:hover {
+  border-color: #eee #eee #ddd;
+}
+.nav-tabs > li.active > a,
+.nav-tabs > li.active > a:hover,
+.nav-tabs > li.active > a:focus {
+  color: #555;
+  cursor: default;
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-bottom-color: transparent;
+}
+.nav-tabs.nav-justified {
+  width: 100%;
+  border-bottom: 0;
+}
+.nav-tabs.nav-justified > li {
+  float: none;
+}
+.nav-tabs.nav-justified > li > a {
+  margin-bottom: 5px;
+  text-align: center;
+}
+.nav-tabs.nav-justified > .dropdown .dropdown-menu {
+  top: auto;
+  left: auto;
+}
+@media (min-width: 768px) {
+  .nav-tabs.nav-justified > li {
+    display: table-cell;
+    width: 1%;
+  }
+  .nav-tabs.nav-justified > li > a {
+    margin-bottom: 0;
+  }
+}
+.nav-tabs.nav-justified > li > a {
+  margin-right: 0;
+  border-radius: 4px;
+}
+.nav-tabs.nav-justified > .active > a,
+.nav-tabs.nav-justified > .active > a:hover,
+.nav-tabs.nav-justified > .active > a:focus {
+  border: 1px solid #ddd;
+}
+@media (min-width: 768px) {
+  .nav-tabs.nav-justified > li > a {
+    border-bottom: 1px solid #ddd;
+    border-radius: 4px 4px 0 0;
+  }
+  .nav-tabs.nav-justified > .active > a,
+  .nav-tabs.nav-justified > .active > a:hover,
+  .nav-tabs.nav-justified > .active > a:focus {
+    border-bottom-color: #fff;
+  }
+}
+.nav-pills > li {
+  float: left;
+}
+.nav-pills > li > a {
+  border-radius: 4px;
+}
+.nav-pills > li + li {
+  margin-left: 2px;
+}
+.nav-pills > li.active > a,
+.nav-pills > li.active > a:hover,
+.nav-pills > li.active > a:focus {
+  color: #fff;
+  background-color: #428bca;
+}
+.nav-stacked > li {
+  float: none;
+}
+.nav-stacked > li + li {
+  margin-top: 2px;
+  margin-left: 0;
+}
+.nav-justified {
+  width: 100%;
+}
+.nav-justified > li {
+  float: none;
+}
+.nav-justified > li > a {
+  margin-bottom: 5px;
+  text-align: center;
+}
+.nav-justified > .dropdown .dropdown-menu {
+  top: auto;
+  left: auto;
+}
+@media (min-width: 768px) {
+  .nav-justified > li {
+    display: table-cell;
+    width: 1%;
+  }
+  .nav-justified > li > a {
+    margin-bottom: 0;
+  }
+}
+.nav-tabs-justified {
+  border-bottom: 0;
+}
+.nav-tabs-justified > li > a {
+  margin-right: 0;
+  border-radius: 4px;
+}
+.nav-tabs-justified > .active > a,
+.nav-tabs-justified > .active > a:hover,
+.nav-tabs-justified > .active > a:focus {
+  border: 1px solid #ddd;
+}
+@media (min-width: 768px) {
+  .nav-tabs-justified > li > a {
+    border-bottom: 1px solid #ddd;
+    border-radius: 4px 4px 0 0;
+  }
+  .nav-tabs-justified > .active > a,
+  .nav-tabs-justified > .active > a:hover,
+  .nav-tabs-justified > .active > a:focus {
+    border-bottom-color: #fff;
+  }
+}
+.tab-content > .tab-pane {
+  display: none;
+}
+.tab-content > .active {
+  display: block;
+}
+.nav-tabs .dropdown-menu {
+  margin-top: -1px;
+  border-top-left-radius: 0;
+  border-top-right-radius: 0;
+}
+.navbar {
+  position: relative;
+  min-height: 50px;
+  margin-bottom: 20px;
+  border: 1px solid transparent;
+}
+@media (min-width: 768px) {
+  .navbar {
+    border-radius: 4px;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-header {
+    float: left;
+  }
+}
+.navbar-collapse {
+  max-height: 340px;
+  padding-right: 15px;
+  padding-left: 15px;
+  overflow-x: visible;
+  -webkit-overflow-scrolling: touch;
+  border-top: 1px solid transparent;
+  box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1);
+}
+.navbar-collapse.in {
+  overflow-y: auto;
+}
+@media (min-width: 768px) {
+  .navbar-collapse {
+    width: auto;
+    border-top: 0;
+    box-shadow: none;
+  }
+  .navbar-collapse.collapse {
+    display: block !important;
+    height: auto !important;
+    padding-bottom: 0;
+    overflow: visible !important;
+  }
+  .navbar-collapse.in {
+    overflow-y: visible;
+  }
+  .navbar-fixed-top .navbar-collapse,
+  .navbar-static-top .navbar-collapse,
+  .navbar-fixed-bottom .navbar-collapse {
+    padding-right: 0;
+    padding-left: 0;
+  }
+}
+.container > .navbar-header,
+.container-fluid > .navbar-header,
+.container > .navbar-collapse,
+.container-fluid > .navbar-collapse {
+  margin-right: -15px;
+  margin-left: -15px;
+}
+@media (min-width: 768px) {
+  .container > .navbar-header,
+  .container-fluid > .navbar-header,
+  .container > .navbar-collapse,
+  .container-fluid > .navbar-collapse {
+    margin-right: 0;
+    margin-left: 0;
+  }
+}
+.navbar-static-top {
+  z-index: 1000;
+  border-width: 0 0 1px;
+}
+@media (min-width: 768px) {
+  .navbar-static-top {
+    border-radius: 0;
+  }
+}
+.navbar-fixed-top,
+.navbar-fixed-bottom {
+  position: fixed;
+  right: 0;
+  left: 0;
+  z-index: 1030;
+}
+@media (min-width: 768px) {
+  .navbar-fixed-top,
+  .navbar-fixed-bottom {
+    border-radius: 0;
+  }
+}
+.navbar-fixed-top {
+  top: 0;
+  border-width: 0 0 1px;
+}
+.navbar-fixed-bottom {
+  bottom: 0;
+  margin-bottom: 0;
+  border-width: 1px 0 0;
+}
+.navbar-brand {
+  float: left;
+  height: 50px;
+  padding: 15px 15px;
+  font-size: 18px;
+  line-height: 20px;
+}
+.navbar-brand:hover,
+.navbar-brand:focus {
+  text-decoration: none;
+}
+@media (min-width: 768px) {
+  .navbar > .container .navbar-brand,
+  .navbar > .container-fluid .navbar-brand {
+    margin-left: -15px;
+  }
+}
+.navbar-toggle {
+  position: relative;
+  float: right;
+  padding: 9px 10px;
+  margin-top: 8px;
+  margin-right: 15px;
+  margin-bottom: 8px;
+  background-color: transparent;
+  background-image: none;
+  border: 1px solid transparent;
+  border-radius: 4px;
+}
+.navbar-toggle:focus {
+  outline: none;
+}
+.navbar-toggle .icon-bar {
+  display: block;
+  width: 22px;
+  height: 2px;
+  border-radius: 1px;
+}
+.navbar-toggle .icon-bar + .icon-bar {
+  margin-top: 4px;
+}
+@media (min-width: 768px) {
+  .navbar-toggle {
+    display: none;
+  }
+}
+.navbar-nav {
+  margin: 7.5px -15px;
+}
+.navbar-nav > li > a {
+  padding-top: 10px;
+  padding-bottom: 10px;
+  line-height: 20px;
+}
+@media (max-width: 767px) {
+  .navbar-nav .open .dropdown-menu {
+    position: static;
+    float: none;
+    width: auto;
+    margin-top: 0;
+    background-color: transparent;
+    border: 0;
+    box-shadow: none;
+  }
+  .navbar-nav .open .dropdown-menu > li > a,
+  .navbar-nav .open .dropdown-menu .dropdown-header {
+    padding: 5px 15px 5px 25px;
+  }
+  .navbar-nav .open .dropdown-menu > li > a {
+    line-height: 20px;
+  }
+  .navbar-nav .open .dropdown-menu > li > a:hover,
+  .navbar-nav .open .dropdown-menu > li > a:focus {
+    background-image: none;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-nav {
+    float: left;
+    margin: 0;
+  }
+  .navbar-nav > li {
+    float: left;
+  }
+  .navbar-nav > li > a {
+    padding-top: 15px;
+    padding-bottom: 15px;
+  }
+  .navbar-nav.navbar-right:last-child {
+    margin-right: -15px;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-left {
+    float: left !important;
+  }
+  .navbar-right {
+    float: right !important;
+  }
+}
+.navbar-form {
+  padding: 10px 15px;
+  margin-top: 8px;
+  margin-right: -15px;
+  margin-bottom: 8px;
+  margin-left: -15px;
+  border-top: 1px solid transparent;
+  border-bottom: 1px solid transparent;
+  -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1);
+          box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1);
+}
+@media (min-width: 768px) {
+  .navbar-form .form-group {
+    display: inline-block;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .navbar-form .form-control {
+    display: inline-block;
+    width: auto;
+    vertical-align: middle;
+  }
+  .navbar-form .input-group > .form-control {
+    width: 100%;
+  }
+  .navbar-form .control-label {
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .navbar-form .radio,
+  .navbar-form .checkbox {
+    display: inline-block;
+    padding-left: 0;
+    margin-top: 0;
+    margin-bottom: 0;
+    vertical-align: middle;
+  }
+  .navbar-form .radio input[type="radio"],
+  .navbar-form .checkbox input[type="checkbox"] {
+    float: none;
+    margin-left: 0;
+  }
+  .navbar-form .has-feedback .form-control-feedback {
+    top: 0;
+  }
+}
+@media (max-width: 767px) {
+  .navbar-form .form-group {
+    margin-bottom: 5px;
+  }
+}
+@media (min-width: 768px) {
+  .navbar-form {
+    width: auto;
+    padding-top: 0;
+    padding-bottom: 0;
+    margin-right: 0;
+    margin-left: 0;
+    border: 0;
+    -webkit-box-shadow: none;
+            box-shadow: none;
+  }
+  .navbar-form.navbar-right:last-child {
+    margin-right: -15px;
+  }
+}
+.navbar-nav > li > .dropdown-menu {
+  margin-top: 0;
+  border-top-left-radius: 0;
+  border-top-right-radius: 0;
+}
+.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu {
+  border-bottom-right-radius: 0;
+  border-bottom-left-radius: 0;
+}
+.navbar-btn {
+  margin-top: 8px;
+  margin-bottom: 8px;
+}
+.navbar-btn.btn-sm {
+  margin-top: 10px;
+  margin-bottom: 10px;
+}
+.navbar-btn.btn-xs {
+  margin-top: 14px;
+  margin-bottom: 14px;
+}
+.navbar-text {
+  margin-top: 15px;
+  margin-bottom: 15px;
+}
+@media (min-width: 768px) {
+  .navbar-text {
+    float: left;
+    margin-right: 15px;
+    margin-left: 15px;
+  }
+  .navbar-text.navbar-right:last-child {
+    margin-right: 0;
+  }
+}
+.navbar-default {
+  background-color: #f8f8f8;
+  border-color: #e7e7e7;
+}
+.navbar-default .navbar-brand {
+  color: #777;
+}
+.navbar-default .navbar-brand:hover,
+.navbar-default .navbar-brand:focus {
+  color: #5e5e5e;
+  background-color: transparent;
+}
+.navbar-default .navbar-text {
+  color: #777;
+}
+.navbar-default .navbar-nav > li > a {
+  color: #777;
+}
+.navbar-default .navbar-nav > li > a:hover,
+.navbar-default .navbar-nav > li > a:focus {
+  color: #333;
+  background-color: transparent;
+}
+.navbar-default .navbar-nav > .active > a,
+.navbar-default .navbar-nav > .active > a:hover,
+.navbar-default .navbar-nav > .active > a:focus {
+  color: #555;
+  background-color: #e7e7e7;
+}
+.navbar-default .navbar-nav > .disabled > a,
+.navbar-default .navbar-nav > .disabled > a:hover,
+.navbar-default .navbar-nav > .disabled > a:focus {
+  color: #ccc;
+  background-color: transparent;
+}
+.navbar-default .navbar-toggle {
+  border-color: #ddd;
+}
+.navbar-default .navbar-toggle:hover,
+.navbar-default .navbar-toggle:focus {
+  background-color: #ddd;
+}
+.navbar-default .navbar-toggle .icon-bar {
+  background-color: #888;
+}
+.navbar-default .navbar-collapse,
+.navbar-default .navbar-form {
+  border-color: #e7e7e7;
+}
+.navbar-default .navbar-nav > .open > a,
+.navbar-default .navbar-nav > .open > a:hover,
+.navbar-default .navbar-nav > .open > a:focus {
+  color: #555;
+  background-color: #e7e7e7;
+}
+@media (max-width: 767px) {
+  .navbar-default .navbar-nav .open .dropdown-menu > li > a {
+    color: #777;
+  }
+  .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover,
+  .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus {
+    color: #333;
+    background-color: transparent;
+  }
+  .navbar-default .navbar-nav .open .dropdown-menu > .active > a,
+  .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover,
+  .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus {
+    color: #555;
+    background-color: #e7e7e7;
+  }
+  .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a,
+  .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover,
+  .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus {
+    color: #ccc;
+    background-color: transparent;
+  }
+}
+.navbar-default .navbar-link {
+  color: #777;
+}
+.navbar-default .navbar-link:hover {
+  color: #333;
+}
+.navbar-inverse {
+  background-color: #222;
+  border-color: #080808;
+}
+.navbar-inverse .navbar-brand {
+  color: #999;
+}
+.navbar-inverse .navbar-brand:hover,
+.navbar-inverse .navbar-brand:focus {
+  color: #fff;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-text {
+  color: #999;
+}
+.navbar-inverse .navbar-nav > li > a {
+  color: #999;
+}
+.navbar-inverse .navbar-nav > li > a:hover,
+.navbar-inverse .navbar-nav > li > a:focus {
+  color: #fff;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-nav > .active > a,
+.navbar-inverse .navbar-nav > .active > a:hover,
+.navbar-inverse .navbar-nav > .active > a:focus {
+  color: #fff;
+  background-color: #080808;
+}
+.navbar-inverse .navbar-nav > .disabled > a,
+.navbar-inverse .navbar-nav > .disabled > a:hover,
+.navbar-inverse .navbar-nav > .disabled > a:focus {
+  color: #444;
+  background-color: transparent;
+}
+.navbar-inverse .navbar-toggle {
+  border-color: #333;
+}
+.navbar-inverse .navbar-toggle:hover,
+.navbar-inverse .navbar-toggle:focus {
+  background-color: #333;
+}
+.navbar-inverse .navbar-toggle .icon-bar {
+  background-color: #fff;
+}
+.navbar-inverse .navbar-collapse,
+.navbar-inverse .navbar-form {
+  border-color: #101010;
+}
+.navbar-inverse .navbar-nav > .open > a,
+.navbar-inverse .navbar-nav > .open > a:hover,
+.navbar-inverse .navbar-nav > .open > a:focus {
+  color: #fff;
+  background-color: #080808;
+}
+@media (max-width: 767px) {
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header {
+    border-color: #080808;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu .divider {
+    background-color: #080808;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > li > a {
+    color: #999;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus {
+    color: #fff;
+    background-color: transparent;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus {
+    color: #fff;
+    background-color: #080808;
+  }
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover,
+  .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus {
+    color: #444;
+    background-color: transparent;
+  }
+}
+.navbar-inverse .navbar-link {
+  color: #999;
+}
+.navbar-inverse .navbar-link:hover {
+  color: #fff;
+}
+.breadcrumb {
+  padding: 8px 15px;
+  margin-bottom: 20px;
+  list-style: none;
+  background-color: #f5f5f5;
+  border-radius: 4px;
+}
+.breadcrumb > li {
+  display: inline-block;
+}
+.breadcrumb > li + li:before {
+  padding: 0 5px;
+  color: #ccc;
+  content: "/\00a0";
+}
+.breadcrumb > .active {
+  color: #999;
+}
+.pagination {
+  display: inline-block;
+  padding-left: 0;
+  margin: 20px 0;
+  border-radius: 4px;
+}
+.pagination > li {
+  display: inline;
+}
+.pagination > li > a,
+.pagination > li > span {
+  position: relative;
+  float: left;
+  padding: 6px 12px;
+  margin-left: -1px;
+  line-height: 1.42857143;
+  color: #428bca;
+  text-decoration: none;
+  background-color: #fff;
+  border: 1px solid #ddd;
+}
+.pagination > li:first-child > a,
+.pagination > li:first-child > span {
+  margin-left: 0;
+  border-top-left-radius: 4px;
+  border-bottom-left-radius: 4px;
+}
+.pagination > li:last-child > a,
+.pagination > li:last-child > span {
+  border-top-right-radius: 4px;
+  border-bottom-right-radius: 4px;
+}
+.pagination > li > a:hover,
+.pagination > li > span:hover,
+.pagination > li > a:focus,
+.pagination > li > span:focus {
+  color: #2a6496;
+  background-color: #eee;
+  border-color: #ddd;
+}
+.pagination > .active > a,
+.pagination > .active > span,
+.pagination > .active > a:hover,
+.pagination > .active > span:hover,
+.pagination > .active > a:focus,
+.pagination > .active > span:focus {
+  z-index: 2;
+  color: #fff;
+  cursor: default;
+  background-color: #428bca;
+  border-color: #428bca;
+}
+.pagination > .disabled > span,
+.pagination > .disabled > span:hover,
+.pagination > .disabled > span:focus,
+.pagination > .disabled > a,
+.pagination > .disabled > a:hover,
+.pagination > .disabled > a:focus {
+  color: #999;
+  cursor: not-allowed;
+  background-color: #fff;
+  border-color: #ddd;
+}
+.pagination-lg > li > a,
+.pagination-lg > li > span {
+  padding: 10px 16px;
+  font-size: 18px;
+}
+.pagination-lg > li:first-child > a,
+.pagination-lg > li:first-child > span {
+  border-top-left-radius: 6px;
+  border-bottom-left-radius: 6px;
+}
+.pagination-lg > li:last-child > a,
+.pagination-lg > li:last-child > span {
+  border-top-right-radius: 6px;
+  border-bottom-right-radius: 6px;
+}
+.pagination-sm > li > a,
+.pagination-sm > li > span {
+  padding: 5px 10px;
+  font-size: 12px;
+}
+.pagination-sm > li:first-child > a,
+.pagination-sm > li:first-child > span {
+  border-top-left-radius: 3px;
+  border-bottom-left-radius: 3px;
+}
+.pagination-sm > li:last-child > a,
+.pagination-sm > li:last-child > span {
+  border-top-right-radius: 3px;
+  border-bottom-right-radius: 3px;
+}
+.pager {
+  padding-left: 0;
+  margin: 20px 0;
+  text-align: center;
+  list-style: none;
+}
+.pager li {
+  display: inline;
+}
+.pager li > a,
+.pager li > span {
+  display: inline-block;
+  padding: 5px 14px;
+  background-color: #fff;
+  border: 1px solid #ddd;
+  border-radius: 15px;
+}
+.pager li > a:hover,
+.pager li > a:focus {
+  text-decoration: none;
+  background-color: #eee;
+}
+.pager .next > a,
+.pager .next > span {
+  float: right;
+}
+.pager .previous > a,
+.pager .previous > span {
+  float: left;
+}
+.pager .disabled > a,
+.pager .disabled > a:hover,
+.pager .disabled > a:focus,
+.pager .disabled > span {
+  color: #999;
+  cursor: not-allowed;
+  background-color: #fff;
+}
+.label {
+  display: inline;
+  padding: .2em .6em .3em;
+  font-size: 75%;
+  font-weight: bold;
+  line-height: 1;
+  color: #fff;
+  text-align: center;
+  white-space: nowrap;
+  vertical-al

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/css/custom.css
----------------------------------------------------------------------
diff --git a/docs/css/custom.css b/docs/css/custom.css
new file mode 100644
index 0000000..3662ec1
--- /dev/null
+++ b/docs/css/custom.css
@@ -0,0 +1,9 @@
+.extLink {
+	display:inline !important; 
+	padding-right:3px !important;
+}
+
+.small-font-awesome {
+	font-size:10px;
+	padding-right: 10px;
+}
\ No newline at end of file


[53/60] Turn Documentation into standalone website, add Overview Page

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/dataset_transformations.md
----------------------------------------------------------------------
diff --git a/docs/dataset_transformations.md b/docs/dataset_transformations.md
index 4fa7ff6..e027eb5 100644
--- a/docs/dataset_transformations.md
+++ b/docs/dataset_transformations.md
@@ -2,23 +2,25 @@
 title: "DataSet Transformations"
 ---
 
-<section id="top">
-DataSet Transformations
------------------------
+* This will be replaced by the TOC
+{:toc}
 
 This document gives a deep-dive into the available transformations on DataSets. For a general introduction to the
-Flink Java API, please refer to the [API guide](java_api_guide.html)
+Flink Java API, please refer to the [Programming Guide](programming_guide.html)
 
 
 ### Map
 
-The Map transformation applies a user-defined `MapFunction` on each element of a DataSet.
+The Map transformation applies a user-defined map function on each element of a DataSet.
 It implements a one-to-one mapping, that is, exactly one element must be returned by
 the function.
 
-The following code transforms a `DataSet` of Integer pairs into a `DataSet` of Integers:
+The following code transforms a DataSet of Integer pairs into a DataSet of Integers:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 // MapFunction that adds two integer values
 public class IntAdder implements MapFunction<Tuple2<Integer, Integer>, Integer> {
   @Override
@@ -30,16 +32,30 @@ public class IntAdder implements MapFunction<Tuple2<Integer, Integer>, Integer>
 // [...]
 DataSet<Tuple2<Integer, Integer>> intPairs = // [...]
 DataSet<Integer> intSums = intPairs.map(new IntAdder());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val intPairs: DataSet[(Int, Int)] = // [...]
+val intSums = intPairs.map { pair => pair._1 + pair._2 }
+~~~
+
+</div>
+</div>
 
 ### FlatMap
 
-The FlatMap transformation applies a user-defined `FlatMapFunction` on each element of a `DataSet`.
+The FlatMap transformation applies a user-defined flat-map function on each element of a DataSet.
 This variant of a map function can return arbitrary many result elements (including none) for each input element.
 
-The following code transforms a `DataSet` of text lines into a `DataSet` of words:
+The following code transforms a DataSet of text lines into a DataSet of words:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 // FlatMapFunction that tokenizes a String by whitespace characters and emits all String tokens.
 public class Tokenizer implements FlatMapFunction<String, String> {
   @Override
@@ -53,17 +69,31 @@ public class Tokenizer implements FlatMapFunction<String, String> {
 // [...]
 DataSet<String> textLines = // [...]
 DataSet<String> words = textLines.flatMap(new Tokenizer());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val textLines: DataSet[String] = // [...]
+val words = textLines.flatMap { _.split(" ") }
+~~~
+
+</div>
+</div>
 
 ### MapPartition
 
-The MapPartition function transforms a parallel partition in a single function call. The function get the partition as an `Iterable` stream and
-can produce an arbitrary number of result values. The number of elements in each partition depends on the degree-of-parallelism
+MapPartition transforms a parallel partition in a single function call. The map-partition function
+gets the partition as Iterable and can produce an arbitrary number of result values. The number of elements in each partition depends on the degree-of-parallelism
 and previous operations.
 
-The following code transforms a `DataSet` of text lines into a `DataSet` of counts per partition:
+The following code transforms a DataSet of text lines into a DataSet of counts per partition:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 public class PartitionCounter implements MapPartitionFunction<String, Long> {
 
   public void mapPartition(Iterable<String> values, Collector<Long> out) {
@@ -78,15 +108,31 @@ public class PartitionCounter implements MapPartitionFunction<String, Long> {
 // [...]
 DataSet<String> textLines = // [...]
 DataSet<Long> counts = textLines.mapPartition(new PartitionCounter());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val textLines: DataSet[String] = // [...]
+// Some is required because the return value must be a Collection.
+// There is an implicit conversion from Option to a Collection.
+val counts = texLines.mapPartition { in => Some(in.size) }
+~~~
+
+</div>
+</div>
 
 ### Filter
 
-The Filter transformation applies a user-defined `FilterFunction` on each element of a `DataSet` and retains only those elements for which the function returns `true`.
+The Filter transformation applies a user-defined filter function on each element of a DataSet and retains only those elements for which the function returns `true`.
+
+The following code removes all Integers smaller than zero from a DataSet:
 
-The following code removes all Integers smaller than zero from a `DataSet`:
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 // FilterFunction that filters out all Integers smaller than zero.
 public class NaturalNumberFilter implements FilterFunction<Integer> {
   @Override
@@ -98,45 +144,64 @@ public class NaturalNumberFilter implements FilterFunction<Integer> {
 // [...]
 DataSet<Integer> intNumbers = // [...]
 DataSet<Integer> naturalNumbers = intNumbers.filter(new NaturalNumberFilter());
-```
+~~~
 
-### Project (Tuple DataSets only)
+</div>
+<div data-lang="scala" markdown="1">
 
-The Project transformation removes or moves `Tuple` fields of a `Tuple` `DataSet`.
-The `project(int...)` method selects `Tuple` fields that should be retained by their index and defines their order in the output `Tuple`.
-The `types(Class<?> ...)`method must give the types of the output `Tuple` fields.
+~~~scala
+val intNumbers: DataSet[Int] = // [...]
+val naturalNumbers = intNumbers.filter { _ > 0 }
+~~~
+
+</div>
+</div>
+
+### Project (Tuple DataSets only) (Java API Only)
+
+The Project transformation removes or moves Tuple fields of a Tuple DataSet.
+The `project(int...)` method selects Tuple fields that should be retained by their index and defines their order in the output Tuple.
+The `types(Class<?> ...)`method must give the types of the output Tuple fields.
 
 Projections do not require the definition of a user function.
 
-The following code shows different ways to apply a Project transformation on a `DataSet`:
+The following code shows different ways to apply a Project transformation on a DataSet:
 
-```java
+~~~java
 DataSet<Tuple3<Integer, Double, String>> in = // [...]
 // converts Tuple3<Integer, Double, String> into Tuple2<String, Integer>
 DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
-```
+~~~
 
-### Transformations on grouped DataSet
+### Transformations on Grouped DataSet
 
 The reduce operations can operate on grouped data sets. Specifying the key to
 be used for grouping can be done in two ways:
 
-- a `KeySelector` function or
-- one or more field position keys (`Tuple` `DataSet` only).
+- a key-selector function or
+- one or more field position keys (Tuple DataSet only).
+- Case Class fields (Case Classes only).
 
 Please look at the reduce examples to see how the grouping keys are specified.
 
-### Reduce on grouped DataSet
+### Reduce on Grouped DataSet
 
-A Reduce transformation that is applied on a grouped `DataSet` reduces each group to a single element using a user-defined `ReduceFunction`.
-For each group of input elements, a `ReduceFunction` successively combines pairs of elements into one element until only a single element for each group remains.
+A Reduce transformation that is applied on a grouped DataSet reduces each group to a single
+element using a user-defined reduce function.
+For each group of input elements, a reduce function successively combines pairs of elements into one
+element until only a single element for each group remains.
 
-#### Reduce on DataSet grouped by KeySelector Function
+#### Reduce on DataSet Grouped by KeySelector Function
 
-A `KeySelector` function extracts a key value from each element of a `DataSet`. The extracted key value is used to group the `DataSet`.
-The following code shows how to group a POJO `DataSet` using a `KeySelector` function and to reduce it with a `ReduceFunction`.
+A key-selector function extracts a key value from each element of a DataSet. The extracted key
+value is used to group the DataSet.
+The following code shows how to group a POJO DataSet using a key-selector function and to reduce it
+with a reduce function.
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 // some ordinary POJO
 public class WC {
   public String word;
@@ -162,14 +227,38 @@ DataSet<WC> wordCounts = words
                            })
                          // apply ReduceFunction on grouped DataSet
                          .reduce(new WordCounter());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+// some ordinary POJO
+class WC(val word: String, val count: Int) {
+  def this() {
+    this(null, -1)
+  }
+  // [...]
+}
+
+val words: DataSet[WC] = // [...]
+val wordCounts = words.groupBy { _.word } reduce { 
+  (w1, w2) => new WC(w1.word, w1.count + w2.count)
+}
+~~~
+
+</div>
+</div>
 
-#### Reduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+#### Reduce on DataSet Grouped by Field Position Keys (Tuple DataSets only)
 
-Field position keys specify one or more fields of a `Tuple` `DataSet` that are used as grouping keys.
-The following code shows how to use field position keys and apply a `ReduceFunction`.
+Field position keys specify one or more fields of a Tuple DataSet that are used as grouping keys.
+The following code shows how to use field position keys and apply a reduce function
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple3<String, Integer, Double>> tuples = // [...]
 DataSet<Tuple3<String, Integer, Double>> reducedTuples =
                                          tuples
@@ -177,19 +266,48 @@ DataSet<Tuple3<String, Integer, Double>> reducedTuples =
                                          .groupBy(0,1)
                                          // apply ReduceFunction on grouped DataSet
                                          .reduce(new MyTupleReducer());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val tuples = DataSet[(String, Int, Double)] = // [...]
+// group on the first and second Tuple field
+val reducedTuples = tuples.groupBy(0, 1).reduce { ... }
+~~~
+
+
+#### Reduce on DataSet grouped by Case Class Fields
+
+When using Case Classes you can also specify the grouping key using the names of the fields: 
 
-### GroupReduce on grouped DataSet
+~~~scala
+case class MyClass(val a: String, b: Int, c: Double)
+val tuples = DataSet[MyClass]] = // [...]
+// group on the first and second field
+val reducedTuples = tuples.groupBy("a", "b").reduce { ... }
+~~~
 
-A GroupReduce transformation that is applied on a grouped `DataSet` calls a user-defined `GroupReduceFunction` for each group. The difference
-between this and `Reduce` is that the user defined function gets the whole group at once.
-The function is invoked with an Iterable over all elements of a group and can return an arbitrary number of result elements using the collector.
+</div>
+</div>
 
-#### GroupReduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+### GroupReduce on Grouped DataSet
 
-The following code shows how duplicate strings can be removed from a `DataSet` grouped by Integer.
+A GroupReduce transformation that is applied on a grouped DataSet calls a user-defined
+group-reduce function for each group. The difference
+between this and *Reduce* is that the user defined function gets the whole group at once.
+The function is invoked with an Iterable over all elements of a group and can return an arbitrary
+number of result elements.
 
-```java
+#### GroupReduce on DataSet Grouped by Field Position Keys (Tuple DataSets only)
+
+The following code shows how duplicate strings can be removed from a DataSet grouped by Integer.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 public class DistinctReduce
          implements GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
 
@@ -217,20 +335,43 @@ DataSet<Tuple2<Integer, String>> input = // [...]
 DataSet<Tuple2<Integer, String>> output = input
                            .groupBy(0)            // group DataSet by the first tuple field
                            .reduceGroup(new DistinctReduce());  // apply GroupReduceFunction
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val input: DataSet[(Int, String)] = // [...]
+val output = input.groupBy(0).reduceGroup {
+      (in, out: Collector[(Int, String)]) =>
+        in.toSet foreach (out.collect)
+    }
+~~~
+
+#### GroupReduce on DataSet Grouped by Case Class Fields
+
+Works analogous to grouping by Case Class fields in *Reduce* transformations.
 
-#### GroupReduce on DataSet grouped by KeySelector Function
 
-Works analogous to `KeySelector` functions in Reduce transformations.
+</div>
+</div>
+
+#### GroupReduce on DataSet Grouped by KeySelector Function
+
+Works analogous to key-selector functions in *Reduce* transformations.
 
 #### GroupReduce on sorted groups (Tuple DataSets only)
 
-A `GroupReduceFunction` accesses the elements of a group using an Iterable. Optionally, the Iterable can hand out the elements of a group in a specified order. In many cases this can help to reduce the complexity of a user-defined `GroupReduceFunction` and improve its efficiency.
+A group-reduce function accesses the elements of a group using an Iterable. Optionally, the Iterable can hand out the elements of a group in a specified order. In many cases this can help to reduce the complexity of a user-defined
+group-reduce function and improve its efficiency.
 Right now, this feature is only available for DataSets of Tuples.
 
-The following code shows another example how to remove duplicate Strings in a `DataSet` grouped by an Integer and sorted by String.
+The following code shows another example how to remove duplicate Strings in a DataSet grouped by an Integer and sorted by String.
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 // GroupReduceFunction that removes consecutive identical elements
 public class DistinctReduce
          implements GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
@@ -259,19 +400,41 @@ DataSet<Double> output = input
                          .groupBy(0)                         // group DataSet by first field
                          .sortGroup(1, Order.ASCENDING)      // sort groups on second tuple field
                          .reduceGroup(new DistinctReduce());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+val input: DataSet[(Int, String)] = // [...]
+val output = input.groupBy(0).sortGroup(1, Order.ASCENDING).reduceGroup {
+      (in, out: Collector[(Int, String)]) =>
+        var prev: (Int, String) = null
+        for (t <- in) {
+          if (prev == null || prev != t)
+            out.collect(t)
+        }
+    }
+
+~~~
+
+</div>
+</div>
 
 **Note:** A GroupSort often comes for free if the grouping is established using a sort-based execution strategy of an operator before the reduce operation.
 
 #### Combinable GroupReduceFunctions
 
-In contrast to a `ReduceFunction`, a `GroupReduceFunction` is not
-necessarily combinable. In order to make a `GroupReduceFunction`
+In contrast to a reduce function, a group-reduce function is not
+necessarily combinable. In order to make a group-reduce function
 combinable, you need to use the `RichGroupReduceFunction` variant,
 implement (override) the `combine()` method, and annotate the
-`GroupReduceFunction` with the `@Combinable` annotation as shown here:
+`RichGroupReduceFunction` with the `@Combinable` annotation as shown here:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 // Combinable GroupReduceFunction that computes two sums.
 // Note that we use the RichGroupReduceFunction because it defines the combine method
 @Combinable
@@ -302,9 +465,49 @@ public class MyCombinableGroupReducer
     this.reduce(in, out);
   }
 }
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+// Combinable GroupReduceFunction that computes two sums.
+// Note that we use the RichGroupReduceFunction because it defines the combine method
+@Combinable
+class MyCombinableGroupReducer
+  extends RichGroupReduceFunction[(String, Int, Double), (String, Int, Double)] {}
+
+  def reduce(
+      in: java.lang.Iterable[(String, Int, Double)],
+      out: Collector[(String, Int, Double)]): Unit = {
+
+    val key: String = null
+    val intSum = 0
+    val doubleSum = 0.0
+
+    for (curr <- in) {
+      key = curr._1
+      intSum += curr._2
+      doubleSum += curr._3
+    }
+    // emit a tuple with both sums
+    out.collect(key, intSum, doubleSum);
+  }
+
+  def combine(
+      in: java.lang.Iterable[(String, Int, Double)],
+      out: Collector[(String, Int, Double)]): Unit = {
+    // in some cases combine() calls can simply be forwarded to reduce().
+    this.reduce(in, out)
+  }
+}
+~~~
+
+</div>
+</div>
 
-### Aggregate on grouped Tuple DataSet
+### Aggregate on Grouped Tuple DataSet
 
 There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
 
@@ -312,17 +515,30 @@ There are some common aggregation operations that are frequently used. The Aggre
 - Min, and
 - Max.
 
-The Aggregate transformation can only be applied on a `Tuple` `DataSet` and supports only field positions keys for grouping.
+The Aggregate transformation can only be applied on a Tuple DataSet and supports only field positions keys for grouping.
 
-The following code shows how to apply an Aggregation transformation on a `DataSet` grouped by field position keys:
+The following code shows how to apply an Aggregation transformation on a DataSet grouped by field position keys:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple3<Integer, String, Double>> input = // [...]
 DataSet<Tuple3<Integer, String, Double>> output = input
                                    .groupBy(1)        // group DataSet on second field
                                    .aggregate(SUM, 0) // compute sum of the first field
                                    .and(MIN, 2);      // compute minimum of the third field
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 To apply multiple aggregations on a DataSet it is necessary to use the `.and()` function after the first aggregate, that means `.aggregate(SUM, 0).and(MIN, 2)` produces the sum of field 0 and the minimum of field 2 of the original DataSet. 
 In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggregation on an aggregation. In the given example it would produce the minimum of field 2 after calculating the sum of field 0 grouped by field 1.
@@ -331,12 +547,15 @@ In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggrega
 
 ### Reduce on full DataSet
 
-The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a `DataSet`.
+The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a DataSet.
 The `ReduceFunction` subsequently combines pairs of elements into one element until only a single element remains.
 
-The following code shows how to sum all elements of an Integer `DataSet`:
+The following code shows how to sum all elements of an Integer DataSet:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 // ReduceFunction that sums Integers
 public class IntSummer implements ReduceFunction<Integer> {
   @Override
@@ -348,24 +567,47 @@ public class IntSummer implements ReduceFunction<Integer> {
 // [...]
 DataSet<Integer> intNumbers = // [...]
 DataSet<Integer> sum = intNumbers.reduce(new IntSummer());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
 
-Reducing a full `DataSet` using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
+~~~
+
+</div>
+</div>
+
+Reducing a full DataSet using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
 
 ### GroupReduce on full DataSet
 
-The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a `DataSet`.
-A `GroupReduceFunction` can iterate over all elements of `DataSet` and return an arbitrary number of result elements.
+The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a DataSet.
+A `GroupReduceFunction` can iterate over all elements of DataSet and return an arbitrary number of result elements.
 
-The following example shows how to apply a GroupReduce transformation on a full `DataSet`:
+The following example shows how to apply a GroupReduce transformation on a full DataSet:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Integer> input = // [...]
 // apply a (preferably combinable) GroupReduceFunction to a DataSet
 DataSet<Double> output = input.reduceGroup(new MyGroupReducer());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
 
-**Note:** A GroupReduce transformation on a full `DataSet` cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
+~~~scala
+
+~~~
+
+</div>
+</div>
+
+**Note:** A GroupReduce transformation on a full DataSet cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
 
 ### Aggregate on full Tuple DataSet
 
@@ -375,35 +617,51 @@ There are some common aggregation operations that are frequently used. The Aggre
 - Min, and
 - Max.
 
-The Aggregate transformation can only be applied on a `Tuple` `DataSet`.
+The Aggregate transformation can only be applied on a Tuple DataSet.
+
+The following code shows how to apply an Aggregation transformation on a full DataSet:
 
-The following code shows how to apply an Aggregation transformation on a full `DataSet`:
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 DataSet<Tuple2<Integer, Double>> input = // [...]
 DataSet<Tuple2<Integer, Double>> output = input
                                      .aggregate(SUM, 0)    // compute sum of the first field
                                      .and(MIN, 1);    // compute minimum of the second field
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 **Note:** Extending the set of supported aggregation functions is on our roadmap.
 
 ### Join
 
-The Join transformation joins two `DataSet`s into one `DataSet`. The elements of both `DataSet`s are joined on one or more keys which can be specified using
+The Join transformation joins two DataSets into one DataSet. The elements of both DataSets are joined on one or more keys which can be specified using
 
 - a `KeySelector` function or
-- one or more field position keys (`Tuple` `DataSet` only).
+- one or more field position keys (Tuple DataSet only).
 
 There are a few different ways to perform a Join transformation which are shown in the following.
 
 #### Default Join (Join into Tuple2)
 
-The default Join transformation produces a new `Tuple``DataSet` with two fields. Each tuple holds a joined element of the first input `DataSet` in the first tuple field and a matching element of the second input `DataSet` in the second field.
+The default Join transformation produces a new TupleDataSet with two fields. Each tuple holds a joined element of the first input DataSet in the first tuple field and a matching element of the second input DataSet in the second field.
 
 The following code shows a default Join transformation using field position keys:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple2<Integer, String>> input1 = // [...]
 DataSet<Tuple2<Double, Integer>> input2 = // [...]
 // result dataset is typed as Tuple2
@@ -411,16 +669,29 @@ DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Double, Integer>>>
             result = input1.join(input2)
                            .where(0)       // key of the first input
                            .equalTo(1);    // key of the second input
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 #### Join with JoinFunction
 
 A Join transformation can also call a user-defined `JoinFunction` to process joining tuples.
-A `JoinFunction` receives one element of the first input `DataSet` and one element of the second input `DataSet` and returns exactly one element.
+A `JoinFunction` receives one element of the first input DataSet and one element of the second input DataSet and returns exactly one element.
+
+The following code performs a join of DataSet with custom java objects and a Tuple DataSet using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
 
-The following code performs a join of `DataSet` with custom java objects and a `Tuple` `DataSet` using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 // some POJO
 public class Rating {
   public String name;
@@ -457,7 +728,17 @@ DataSet<Tuple2<String, Double>>
 
                    // applying the JoinFunction on joining pairs
                    .with(new PointWeighter());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 #### Join with FlatJoinFunction
 
@@ -485,7 +766,10 @@ DataSet<Tuple2<String, Double>>
 
 A Join transformation can construct result tuples using a projection as shown here:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
 DataSet<Tuple2<Integer, Double>> input2 = // [...]
 DataSet<Tuple4<Integer, String, Double, Byte>
@@ -498,16 +782,29 @@ DataSet<Tuple4<Integer, String, Double, Byte>
                   // select and reorder fields of matching tuples
                   .projectFirst(0,2).projectSecond(1).projectFirst(1)
                   .types(Integer.class, String.class, Double.class, Byte.class);
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
 
-`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output `Tuple`. The order of indexes defines the order of fields in the output tuple.
-The join projection works also for non-`Tuple` `DataSet`s. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output `Tuple`.
+~~~scala
+
+~~~
+
+</div>
+</div>
+
+`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output Tuple. The order of indexes defines the order of fields in the output tuple.
+The join projection works also for non-Tuple DataSets. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output Tuple.
 
 #### Join with DataSet Size Hint
 
-In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to join as shown here:
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a DataSet to join as shown here:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 DataSet<Tuple2<Integer, String>> input1 = // [...]
 DataSet<Tuple2<Integer, String>> input2 = // [...]
 
@@ -524,11 +821,21 @@ DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
             input1.joinWithHuge(input2)
                   .where(0)
                   .equalTo(0);
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 ### Cross
 
-The Cross transformation combines two `DataSet`s into one `DataSet`. It builds all pairwise combinations of the elements of both input `DataSet`s, i.e., it builds a Cartesian product.
+The Cross transformation combines two DataSets into one DataSet. It builds all pairwise combinations of the elements of both input DataSets, i.e., it builds a Cartesian product.
 The Cross transformation either calls a user-defined `CrossFunction` on each pair of elements or applies a projection. Both modes are shown in the following.
 
 **Note:** Cross is potentially a *very* compute-intensive operation which can challenge even large compute clusters!
@@ -537,9 +844,12 @@ The Cross transformation either calls a user-defined `CrossFunction` on each pai
 
 A Cross transformation can call a user-defined `CrossFunction`. A `CrossFunction` receives one element of the first input and one element of the second input and returns exactly one result element.
 
-The following code shows how to apply a Cross transformation on two `DataSet`s using a `CrossFunction`:
+The following code shows how to apply a Cross transformation on two DataSets using a `CrossFunction`:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 public class Coord {
   public int id;
   public int x;
@@ -565,13 +875,26 @@ DataSet<Tuple3<Integer, Integer, Double>>
             coords1.cross(coords2)
                    // apply CrossFunction
                    .with(new EuclideanDistComputer());
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 #### Cross with Projection
 
 A Cross transformation can also construct result tuples using a projection as shown here:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
 DataSet<Tuple2<Integer, Double>> input2 = // [...]
 DataSet<Tuple4<Integer, Byte, Integer, Double>
@@ -580,15 +903,28 @@ DataSet<Tuple4<Integer, Byte, Integer, Double>
                   // select and reorder fields of matching tuples
                   .projectSecond(0).projectFirst(1,0).projectSecond(1)
                   .types(Integer.class, Byte.class, Integer.class, Double.class);
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 The field selection in a Cross projection works the same way as in the projection of Join results.
 
 #### Cross with DataSet Size Hint
 
-In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to cross as shown here:
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a DataSet to cross as shown here:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple2<Integer, String>> input1 = // [...]
 DataSet<Tuple2<Integer, String>> input2 = // [...]
 
@@ -605,21 +941,34 @@ DataSet<Tuple3<Integer, Integer, String>>
             input1.crossWithHuge(input2)
                   // apply a projection (or any Cross function)
                   .projectFirst(0,1).projectSecond(1).types(Integer.class, String.class, String.class)
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
+
+</div>
+</div>
 
 ### CoGroup
 
-The CoGroup transformation jointly processes groups of two `DataSet`s. Both `DataSet`s are grouped on a defined key and groups of both `DataSet`s that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one `DataSet` has a group, the `CoGroupFunction` is called with this group and an empty group.
+The CoGroup transformation jointly processes groups of two DataSets. Both DataSets are grouped on a defined key and groups of both DataSets that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one DataSet has a group, the `CoGroupFunction` is called with this group and an empty group.
 A `CoGroupFunction` can separately iterate over the elements of both groups and return an arbitrary number of result elements.
 
 Similar to Reduce, GroupReduce, and Join, keys can be defined using
 
 - a `KeySelector` function or
-- one or more field position keys (`Tuple` `DataSet` only).
+- one or more field position keys (Tuple DataSet only).
+
+#### CoGroup on DataSets Grouped by Field Position Keys (Tuple DataSets only)
 
-#### CoGroup on DataSets grouped by Field Position Keys (Tuple DataSets only)
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 
-```java
+~~~java
 // Some CoGroupFunction definition
 class MyCoGrouper
          implements CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Double>, Double> {
@@ -655,23 +1004,44 @@ DataSet<Double> output = iVals.coGroup(dVals)
                          .equalTo(0)
                          // apply CoGroup function on each pair of groups
                          .with(new MyCoGrouper());
-```
+~~~
 
-#### CoGroup on DataSets grouped by Key Selector Function
+</div>
+<div data-lang="scala" markdown="1">
 
-Works analogous to key selector functions in Join transformations.
+~~~scala
+
+~~~
+
+</div>
+</div>
+
+#### CoGroup on DataSets Grouped by Key-Selector Function
+
+Works analogous to key-selector functions in Join transformations.
 
 ### Union
 
-Produces the union of two `DataSet`s, which have to be of the same type. A union of more than two `DataSet`s can be implemented with multiple union calls, as shown here:
+Produces the union of two DataSets, which have to be of the same type. A union of more than two DataSets can be implemented with multiple union calls, as shown here:
 
-```java
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+
+~~~java
 DataSet<Tuple2<String, Integer>> vals1 = // [...]
 DataSet<Tuple2<String, Integer>> vals2 = // [...]
 DataSet<Tuple2<String, Integer>> vals3 = // [...]
 DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2)
                     .union(vals3);
-```
+~~~
+
+</div>
+<div data-lang="scala" markdown="1">
+
+~~~scala
+
+~~~
 
+</div>
+</div>
 
-[Back to top](#top)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/hadoop_compatability.md
----------------------------------------------------------------------
diff --git a/docs/hadoop_compatability.md b/docs/hadoop_compatability.md
deleted file mode 100644
index 06c0dfa..0000000
--- a/docs/hadoop_compatability.md
+++ /dev/null
@@ -1,5 +0,0 @@
----
-title: "Hadoop Compatability"
----
-
-To be written.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/hadoop_compatibility.md
----------------------------------------------------------------------
diff --git a/docs/hadoop_compatibility.md b/docs/hadoop_compatibility.md
new file mode 100644
index 0000000..06c0dfa
--- /dev/null
+++ b/docs/hadoop_compatibility.md
@@ -0,0 +1,5 @@
+---
+title: "Hadoop Compatability"
+---
+
+To be written.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/img/apache-incubator-logo.png
----------------------------------------------------------------------
diff --git a/docs/img/apache-incubator-logo.png b/docs/img/apache-incubator-logo.png
new file mode 100644
index 0000000..81fb31e
Binary files /dev/null and b/docs/img/apache-incubator-logo.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index 46dc66c..d6da670 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -1,5 +1,23 @@
 ---
-layout: docs
-title: Apache Flink Documentation
+title: Flink Overview
 ---
 
+Apache Flink (incubating) is a platform for efficient, distributed, general-purpose data processing.
+It features powerful programming abstractions in Java and Scala, a high-performance runtime, and
+automatic program optimization. It has native support for iterations, incremental iterations, and
+programs consisting of large DAGs of operations.
+
+If you quickly want to try out the system, please look at one of the available quickstarts. For
+a thorough introduction of the Flink API please refer to the
+[Programming Guide](programming_guide.htmls).
+
+## Download
+
+You can download Flink from the [downloads]({{ site.FLINK_DOWNLOAD_URL }}) page
+of the [project website]({{ site.FLINK_WEBSITE_URL }}). This documentation if for version {{ site.FLINK_VERSION_STABLE }}. Be careful
+when picking a version, there are different versions depending on the Hadoop and/or
+HDFS version that you want to use with Flink. Please refer to [building](building.html) if you
+want to build Flink yourself from the source.
+
+In Version {{ site.FLINK_VERSION}} the Scala API uses {{ site.SCALA_VERSION_SHORT}}. Please make
+sure to use a compatible version.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/js/bootstrap.min.js
----------------------------------------------------------------------
diff --git a/docs/js/bootstrap.min.js b/docs/js/bootstrap.min.js
new file mode 100644
index 0000000..b04a0e8
--- /dev/null
+++ b/docs/js/bootstrap.min.js
@@ -0,0 +1,6 @@
+/*!
+ * Bootstrap v3.1.1 (http://getbootstrap.com)
+ * Copyright 2011-2014 Twitter, Inc.
+ * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
+ */
+if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one(a.support.transition.end,function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b()})}(jQuery),+function(a){"use strict";var b='[data-dismiss="alert"]',c=function(c){a(c).on("click",b,this.close)};c.prototype.close=function(b){function c(){f.trigger("closed.bs.alert").remove()}var d=a(this),e=d.attr("data-target");e||(e=d.attr("href"),e=e&&e.replace(/.*(?=#[^\s]*$)/,""));var f=a(e);b&&b.preventDefault(),f.length||(f=d.hasClass("alert")?d:d.parent()),f.trigger(b=a.
 Event("close.bs.alert")),b.isDefaultPrevented()||(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one(a.support.transition.end,c).emulateTransitionEnd(150):c())};var d=a.fn.alert;a.fn.alert=function(b){return this.each(function(){var d=a(this),e=d.data("bs.alert");e||d.data("bs.alert",e=new c(this)),"string"==typeof b&&e[b].call(d)})},a.fn.alert.Constructor=c,a.fn.alert.noConflict=function(){return a.fn.alert=d,this},a(document).on("click.bs.alert.data-api",b,c.prototype.close)}(jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.isLoading=!1};b.DEFAULTS={loadingText:"loading..."},b.prototype.setState=function(b){var c="disabled",d=this.$element,e=d.is("input")?"val":"html",f=d.data();b+="Text",f.resetText||d.data("resetText",d[e]()),d[e](f[b]||this.options[b]),setTimeout(a.proxy(function(){"loadingText"==b?(this.isLoading=!0,d.addClass(c).attr(c,c)):this.isLoading&&(this.isLoading=!1,d.removeClass(c).re
 moveAttr(c))},this),0)},b.prototype.toggle=function(){var a=!0,b=this.$element.closest('[data-toggle="buttons"]');if(b.length){var c=this.$element.find("input");"radio"==c.prop("type")&&(c.prop("checked")&&this.$element.hasClass("active")?a=!1:b.find(".active").removeClass("active")),a&&c.prop("checked",!this.$element.hasClass("active")).trigger("change")}a&&this.$element.toggleClass("active")};var c=a.fn.button;a.fn.button=function(c){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof c&&c;e||d.data("bs.button",e=new b(this,f)),"toggle"==c?e.toggle():c&&e.setState(c)})},a.fn.button.Constructor=b,a.fn.button.noConflict=function(){return a.fn.button=c,this},a(document).on("click.bs.button.data-api","[data-toggle^=button]",function(b){var c=a(b.target);c.hasClass("btn")||(c=c.closest(".btn")),c.button("toggle"),b.preventDefault()})}(jQuery),+function(a){"use strict";var b=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-in
 dicators"),this.options=c,this.paused=this.sliding=this.interval=this.$active=this.$items=null,"hover"==this.options.pause&&this.$element.on("mouseenter",a.proxy(this.pause,this)).on("mouseleave",a.proxy(this.cycle,this))};b.DEFAULTS={interval:5e3,pause:"hover",wrap:!0},b.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},b.prototype.getActiveIndex=function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},b.prototype.to=function(b){var c=this,d=this.getActiveIndex();return b>this.$items.length-1||0>b?void 0:this.sliding?this.$element.one("slid.bs.carousel",function(){c.to(b)}):d==b?this.pause().cycle():this.slide(b>d?"next":"prev",a(this.$items[b]))},b.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .p
 rev").length&&a.support.transition&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},b.prototype.next=function(){return this.sliding?void 0:this.slide("next")},b.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},b.prototype.slide=function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g="next"==b?"left":"right",h="next"==b?"first":"last",i=this;if(!e.length){if(!this.options.wrap)return;e=this.$element.find(".item")[h]()}if(e.hasClass("active"))return this.sliding=!1;var j=a.Event("slide.bs.carousel",{relatedTarget:e[0],direction:g});return this.$element.trigger(j),j.isDefaultPrevented()?void 0:(this.sliding=!0,f&&this.pause(),this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid.bs.carousel",function(){var b=a(i.$indicators.children()[i.getActiveIndex()]);b&&b.addClass("active")})),a.support.transition&&this.$element.h
 asClass("slide")?(e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),d.one(a.support.transition.end,function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].join(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger("slid.bs.carousel")},0)}).emulateTransitionEnd(1e3*d.css("transition-duration").slice(0,-1))):(d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger("slid.bs.carousel")),f&&this.cycle(),this)};var c=a.fn.carousel;a.fn.carousel=function(c){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c),g="string"==typeof c?c:f.slide;e||d.data("bs.carousel",e=new b(this,f)),"number"==typeof c?e.to(c):g?e[g]():f.interval&&e.pause().cycle()})},a.fn.carousel.Constructor=b,a.fn.carousel.noConflict=function(){return a.fn.carousel=c,this},a(document).on("click.bs.carousel.data-api","[data-slide], [data-slide-to]",function(b){var c,d=a(this),e=a(d.a
 ttr("data-target")||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"")),f=a.extend({},e.data(),d.data()),g=d.attr("data-slide-to");g&&(f.interval=!1),e.carousel(f),(g=d.attr("data-slide-to"))&&e.data("bs.carousel").to(g),b.preventDefault()}),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var b=a(this);b.carousel(b.data())})})}(jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.transitioning=null,this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};b.DEFAULTS={toggle:!0},b.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},b.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b=a.Event("show.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.$parent&&this.$parent.find("> .panel > .in");if(c&&c.length){var d=c.data("bs.collapse");if(d&&d.transition
 ing)return;c.collapse("hide"),d||c.data("bs.collapse",null)}var e=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[e](0),this.transitioning=1;var f=function(){this.$element.removeClass("collapsing").addClass("collapse in")[e]("auto"),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return f.call(this);var g=a.camelCase(["scroll",e].join("-"));this.$element.one(a.support.transition.end,a.proxy(f,this)).emulateTransitionEnd(350)[e](this.$element[0][g])}}},b.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse").removeClass("in"),this.transitioning=1;var d=function(){this.transitioning=0,this.$element.trigger("hidden.bs.collapse").removeClass("collapsing").addClass("collap
 se")};return a.support.transition?void this.$element[c](0).one(a.support.transition.end,a.proxy(d,this)).emulateTransitionEnd(350):d.call(this)}}},b.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":"show"]()};var c=a.fn.collapse;a.fn.collapse=function(c){return this.each(function(){var d=a(this),e=d.data("bs.collapse"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c);!e&&f.toggle&&"show"==c&&(c=!c),e||d.data("bs.collapse",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.collapse.Constructor=b,a.fn.collapse.noConflict=function(){return a.fn.collapse=c,this},a(document).on("click.bs.collapse.data-api","[data-toggle=collapse]",function(b){var c,d=a(this),e=d.attr("data-target")||b.preventDefault()||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,""),f=a(e),g=f.data("bs.collapse"),h=g?"toggle":d.data(),i=d.attr("data-parent"),j=i&&a(i);g&&g.transitioning||(j&&j.find('[data-toggle=collapse][data-parent="'+i+'"]').not(d).addClass("collapsed"),d[f.hasClass("in
 ")?"addClass":"removeClass"]("collapsed")),f.collapse(h)})}(jQuery),+function(a){"use strict";function b(b){a(d).remove(),a(e).each(function(){var d=c(a(this)),e={relatedTarget:this};d.hasClass("open")&&(d.trigger(b=a.Event("hide.bs.dropdown",e)),b.isDefaultPrevented()||d.removeClass("open").trigger("hidden.bs.dropdown",e))})}function c(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#[A-Za-z]/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}var d=".dropdown-backdrop",e="[data-toggle=dropdown]",f=function(b){a(b).on("click.bs.dropdown",this.toggle)};f.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=c(e),g=f.hasClass("open");if(b(),!g){"ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a('<div class="dropdown-backdrop"/>').insertAfter(a(this)).on("click",b);var h={relatedTarget:this};if(f.trigger(d=a.Event("show.bs.dropdown",h)),d.isDefaultPrevented())return;f.toggleClass("op
 en").trigger("shown.bs.dropdown",h),e.focus()}return!1}},f.prototype.keydown=function(b){if(/(38|40|27)/.test(b.keyCode)){var d=a(this);if(b.preventDefault(),b.stopPropagation(),!d.is(".disabled, :disabled")){var f=c(d),g=f.hasClass("open");if(!g||g&&27==b.keyCode)return 27==b.which&&f.find(e).focus(),d.click();var h=" li:not(.divider):visible a",i=f.find("[role=menu]"+h+", [role=listbox]"+h);if(i.length){var j=i.index(i.filter(":focus"));38==b.keyCode&&j>0&&j--,40==b.keyCode&&j<i.length-1&&j++,~j||(j=0),i.eq(j).focus()}}}};var g=a.fn.dropdown;a.fn.dropdown=function(b){return this.each(function(){var c=a(this),d=c.data("bs.dropdown");d||c.data("bs.dropdown",d=new f(this)),"string"==typeof b&&d[b].call(c)})},a.fn.dropdown.Constructor=f,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=g,this},a(document).on("click.bs.dropdown.data-api",b).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",e,f.prototype.toggle).on(
 "keydown.bs.dropdown.data-api",e+", [role=menu], [role=listbox]",f.prototype.keydown)}(jQuery),+function(a){"use strict";var b=function(b,c){this.options=c,this.$element=a(b),this.$backdrop=this.isShown=null,this.options.remote&&this.$element.find(".modal-content").load(this.options.remote,a.proxy(function(){this.$element.trigger("loaded.bs.modal")},this))};b.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},b.prototype.toggle=function(a){return this[this.isShown?"hide":"show"](a)},b.prototype.show=function(b){var c=this,d=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(d),this.isShown||d.isDefaultPrevented()||(this.isShown=!0,this.escape(),this.$element.on("click.dismiss.bs.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.backdrop(function(){var d=a.support.transition&&c.$element.hasClass("fade");c.$element.parent().length||c.$element.appendTo(document.body),c.$element.show().scrollTop(0),d&&c.$element[0].offsetWidth,c.$element.addClass("in").attr("aria-hidde
 n",!1),c.enforceFocus();var e=a.Event("shown.bs.modal",{relatedTarget:b});d?c.$element.find(".modal-dialog").one(a.support.transition.end,function(){c.$element.focus().trigger(e)}).emulateTransitionEnd(300):c.$element.focus().trigger(e)}))},b.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),a(document).off("focusin.bs.modal"),this.$element.removeClass("in").attr("aria-hidden",!0).off("click.dismiss.bs.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one(a.support.transition.end,a.proxy(this.hideModal,this)).emulateTransitionEnd(300):this.hideModal())},b.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.focus()},this))},b.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.o
 n("keyup.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keyup.dismiss.bs.modal")},b.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.removeBackdrop(),a.$element.trigger("hidden.bs.modal")})},b.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},b.prototype.backdrop=function(b){var c=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var d=a.support.transition&&c;if(this.$backdrop=a('<div class="modal-backdrop '+c+'" />').appendTo(document.body),this.$element.on("click.dismiss.bs.modal",a.proxy(function(a){a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus.call(this.$element[0]):this.hide.call(this))},this)),d&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;d?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()}else!this.isShown&&this.
 $backdrop?(this.$backdrop.removeClass("in"),a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()):b&&b()};var c=a.fn.modal;a.fn.modal=function(c,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},b.DEFAULTS,e.data(),"object"==typeof c&&c);f||e.data("bs.modal",f=new b(this,g)),"string"==typeof c?f[c](d):g.show&&f.show(d)})},a.fn.modal.Constructor=b,a.fn.modal.noConflict=function(){return a.fn.modal=c,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(b){var c=a(this),d=c.attr("href"),e=a(c.attr("data-target")||d&&d.replace(/.*(?=#[^\s]+$)/,"")),f=e.data("bs.modal")?"toggle":a.extend({remote:!/#/.test(d)&&d},e.data(),c.data());c.is("a")&&b.preventDefault(),e.modal(f,this).one("hide",function(){c.is(":visible")&&c.focus()})}),a(document).on("show.bs.modal",".modal",function(){a(document.body).addClass("modal-open")}).on("hidden.bs.modal",".modal",functi
 on(){a(document.body).removeClass("modal-open")})}(jQuery),+function(a){"use strict";var b=function(a,b){this.type=this.options=this.enabled=this.timeout=this.hoverState=this.$element=null,this.init("tooltip",a,b)};b.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},b.prototype.init=function(b,c,d){this.enabled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d);for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focusin",i="hover"==g?"mouseleave":"focusout";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector
 ?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},b.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},b.prototype.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?void(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show)):c.show()},b.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="o
 ut",c.options.delay&&c.options.delay.hide?void(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide)):c.hide()},b.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){if(this.$element.trigger(b),b.isDefaultPrevented())return;var c=this,d=this.tip();this.setContent(),this.options.animation&&d.addClass("fade");var e="function"==typeof this.options.placement?this.options.placement.call(this,d[0],this.$element[0]):this.options.placement,f=/\s?auto?\s?/i,g=f.test(e);g&&(e=e.replace(f,"")||"top"),d.detach().css({top:0,left:0,display:"block"}).addClass(e),this.options.container?d.appendTo(this.options.container):d.insertAfter(this.$element);var h=this.getPosition(),i=d[0].offsetWidth,j=d[0].offsetHeight;if(g){var k=this.$element.parent(),l=e,m=document.documentElement.scrollTop||document.body.scrollTop,n="body"==this.options.container?window.innerWidth:k.outerWidth(),o="body"==this.options.container?window.innerHei
 ght:k.outerHeight(),p="body"==this.options.container?0:k.offset().left;e="bottom"==e&&h.top+h.height+j-m>o?"top":"top"==e&&h.top-m-j<0?"bottom":"right"==e&&h.right+i>n?"left":"left"==e&&h.left-i<p?"right":e,d.removeClass(l).addClass(e)}var q=this.getCalculatedOffset(e,h,i,j);this.applyPlacement(q,e),this.hoverState=null;var r=function(){c.$element.trigger("shown.bs."+c.type)};a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,r).emulateTransitionEnd(150):r()}},b.prototype.applyPlacement=function(b,c){var d,e=this.tip(),f=e[0].offsetWidth,g=e[0].offsetHeight,h=parseInt(e.css("margin-top"),10),i=parseInt(e.css("margin-left"),10);isNaN(h)&&(h=0),isNaN(i)&&(i=0),b.top=b.top+h,b.left=b.left+i,a.offset.setOffset(e[0],a.extend({using:function(a){e.css({top:Math.round(a.top),left:Math.round(a.left)})}},b),0),e.addClass("in");var j=e[0].offsetWidth,k=e[0].offsetHeight;if("top"==c&&k!=g&&(d=!0,b.top=b.top+g-k),/bottom|top/.test(c)){var l=0;b.left<0&&(l=-2*b.left,b
 .left=0,e.offset(b),j=e[0].offsetWidth,k=e[0].offsetHeight),this.replaceArrow(l-f+j,j,"left")}else this.replaceArrow(k-g,k,"top");d&&e.offset(b)},b.prototype.replaceArrow=function(a,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},b.prototype.hide=function(){function b(){"in"!=c.hoverState&&d.detach(),c.$element.trigger("hidden.bs."+c.type)}var c=this,d=this.tip(),e=a.Event("hide.bs."+this.type);return this.$element.trigger(e),e.isDefaultPrevented()?void 0:(d.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,b).emulateTransitionEnd(150):b(),this.hoverState=null,this)},b.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},b.prototype.
 hasContent=function(){return this.getTitle()},b.prototype.getPosition=function(){var b=this.$element[0];return a.extend({},"function"==typeof b.getBoundingClientRect?b.getBoundingClientRect():{width:b.offsetWidth,height:b.offsetHeight},this.$element.offset())},b.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},b.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},b.prototype.tip=function(){return this.$tip=this.$tip||a(this.options.template)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},b.prototype.validate=function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},b.prototype.enabl
 e=function(){this.enabled=!0},b.prototype.disable=function(){this.enabled=!1},b.prototype.toggleEnabled=function(){this.enabled=!this.enabled},b.prototype.toggle=function(b){var c=b?a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type):this;c.tip().hasClass("in")?c.leave(c):c.enter(c)},b.prototype.destroy=function(){clearTimeout(this.timeout),this.hide().$element.off("."+this.type).removeData("bs."+this.type)};var c=a.fn.tooltip;a.fn.tooltip=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof c&&c;(e||"destroy"!=c)&&(e||d.data("bs.tooltip",e=new b(this,f)),"string"==typeof c&&e[c]())})},a.fn.tooltip.Constructor=b,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=c,this}}(jQuery),+function(a){"use strict";var b=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");b.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content
 :"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),b.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),b.prototype.constructor=b,b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content")[this.options.html?"string"==typeof c?"html":"append":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},b.prototype.hasContent=function(){return this.getTitle()||this.getContent()},b.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("function"==typeof b.content?b.content.call(a[0]):b.content)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")},b.prototype.tip=function(){return this.
 $tip||(this.$tip=a(this.options.template)),this.$tip};var c=a.fn.popover;a.fn.popover=function(c){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof c&&c;(e||"destroy"!=c)&&(e||d.data("bs.popover",e=new b(this,f)),"string"==typeof c&&e[c]())})},a.fn.popover.Constructor=b,a.fn.popover.noConflict=function(){return a.fn.popover=c,this}}(jQuery),+function(a){"use strict";function b(c,d){var e,f=a.proxy(this.process,this);this.$element=a(a(c).is("body")?window:c),this.$body=a("body"),this.$scrollElement=this.$element.on("scroll.bs.scroll-spy.data-api",f),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||(e=a(c).attr("href"))&&e.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.offsets=a([]),this.targets=a([]),this.activeTarget=null,this.refresh(),this.process()}b.DEFAULTS={offset:10},b.prototype.refresh=function(){var b=this.$element[0]==window?"offset":"position";this.offsets=a([]),this.targets=a([]);{var c=this;this.$body.fi
 nd(this.selector).map(function(){var d=a(this),e=d.data("target")||d.attr("href"),f=/^#./.test(e)&&a(e);return f&&f.length&&f.is(":visible")&&[[f[b]().top+(!a.isWindow(c.$scrollElement.get(0))&&c.$scrollElement.scrollTop()),e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){c.offsets.push(this[0]),c.targets.push(this[1])})}},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,d=c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(b>=d)return g!=(a=f.last()[0])&&this.activate(a);if(g&&b<=e[0])return g!=(a=f[0])&&this.activate(a);for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(!e[a+1]||b<=e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,a(this.selector).parentsUntil(this.options.target,".active").removeClass("active");var c=this.selector+'[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li"
 ).addClass("active");d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")};var c=a.fn.scrollspy;a.fn.scrollspy=function(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=c,this},a(window).on("load",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);b.scrollspy(b.data())})})}(jQuery),+function(a){"use strict";var b=function(b){this.element=a(b)};b.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a")[0],f=a.Event("show.bs.tab",{relatedTarget:e});if(b.trigger(f),!f.isDefaultPrevented()){var g=a(d);this.activate(b.parent("li"),c),this
 .activate(g,g.parent(),function(){b.trigger({type:"shown.bs.tab",relatedTarget:e})})}}},b.prototype.activate=function(b,c,d){function e(){f.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),g?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var f=c.find("> .active"),g=d&&a.support.transition&&f.hasClass("fade");g?f.one(a.support.transition.end,e).emulateTransitionEnd(150):e(),f.removeClass("in")};var c=a.fn.tab;a.fn.tab=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new b(this)),"string"==typeof c&&e[c]()})},a.fn.tab.Constructor=b,a.fn.tab.noConflict=function(){return a.fn.tab=c,this},a(document).on("click.bs.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(b){b.preventDefault(),a(this).tab("show")})}(jQuery),+function(a){"use strict";var b=function(c,d){this.options=a.extend({},b.DE
 FAULTS,d),this.$window=a(window).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(c),this.affixed=this.unpin=this.pinnedOffset=null,this.checkPosition()};b.RESET="affix affix-top affix-bottom",b.DEFAULTS={offset:0},b.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(b.RESET).addClass("affix");var a=this.$window.scrollTop(),c=this.$element.offset();return this.pinnedOffset=c.top-a},b.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},b.prototype.checkPosition=function(){if(this.$element.is(":visible")){var c=a(document).height(),d=this.$window.scrollTop(),e=this.$element.offset(),f=this.options.offset,g=f.top,h=f.bottom;"top"==this.affixed&&(e.top+=d),"object"!=typeof f&&(h=g=f),"function"==typeof g&&(g=f.top(this.$element)),"function"==typeof h&&(h=f.bottom(this.$element));va
 r i=null!=this.unpin&&d+this.unpin<=e.top?!1:null!=h&&e.top+this.$element.height()>=c-h?"bottom":null!=g&&g>=d?"top":!1;if(this.affixed!==i){this.unpin&&this.$element.css("top","");var j="affix"+(i?"-"+i:""),k=a.Event(j+".bs.affix");this.$element.trigger(k),k.isDefaultPrevented()||(this.affixed=i,this.unpin="bottom"==i?this.getPinnedOffset():null,this.$element.removeClass(b.RESET).addClass(j).trigger(a.Event(j.replace("affix","affixed"))),"bottom"==i&&this.$element.offset({top:c-h-this.$element.height()}))}}};var c=a.fn.affix;a.fn.affix=function(c){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof c&&c;e||d.data("bs.affix",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.affix.Constructor=b,a.fn.affix.noConflict=function(){return a.fn.affix=c,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var b=a(this),c=b.data();c.offset=c.offset||{},c.offsetBottom&&(c.offset.bottom=c.offsetBottom),c.offsetTop&&(c.offset.top=c.offsetT
 op),b.affix(c)})})}(jQuery);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8a8780f/docs/programming_guide.md
----------------------------------------------------------------------
diff --git a/docs/programming_guide.md b/docs/programming_guide.md
index 883e769..3180692 100644
--- a/docs/programming_guide.md
+++ b/docs/programming_guide.md
@@ -1235,7 +1235,7 @@ wordCounts groupBy { _.word } reduce(new MyReduceFunction())
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 
-You can use the `Tuple` classes for composite types. Tuples contain a fix number of fields of
+You can use the Tuple classes for composite types. Tuples contain a fix number of fields of
 various types. The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple
 can be an arbitrary Flink type - including further tuples, resulting in nested tuples. Fields of a
 Tuple can be accessed directly using the fields `tuple.f4`, or using the generic getter method
@@ -1266,7 +1266,7 @@ wordCounts
 {% endhighlight %}
 
 In order to access fields more intuitively and to generate more readable code, it is also possible
-to extend a subclass of `Tuple`. You can add getters and setters with custom names that delegate to
+to extend a subclass of Tuple. You can add getters and setters with custom names that delegate to
 the field positions. See this
 {% gh_link /flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java "example" %} for an
 illustration how to make use of that mechanism.


[60/60] git commit: [scala] Add package documentation for package api.scala

Posted by al...@apache.org.
[scala] Add package documentation for package api.scala


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

Branch: refs/heads/master
Commit: ec562ceb57364a75437e5a93c32aa061f94708d8
Parents: 0c79538
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Sep 20 18:00:23 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 13:43:03 2014 +0200

----------------------------------------------------------------------
 .../main/scala/org/apache/flink/api/scala/package.scala | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ec562ceb/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
index 8bb69f9..998d579 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -24,6 +24,18 @@ import org.apache.flink.types.TypeInformation
 import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
 
+/**
+ * The Flink Scala API. [[org.apache.flink.api.scala.ExecutionEnvironment]] is the starting-point
+ * of any Flink program. It can be used to read from local files, HDFS, or other sources.
+ * [[org.apache.flink.api.scala.DataSet]] is the main abstraction of data in Flink. It provides
+ * operations that create new DataSets via transformations.
+ * [[org.apache.flink.api.scala.GroupedDataSet]] provides operations on grouped data that results
+ * from [[org.apache.flink.api.scala.DataSet.groupBy()]].
+ *
+ * Use [[org.apache.flink.api.scala.ExecutionEnvironment.getExecutionEnvironment]] to obtain
+ * an execution environment. This will either create a local environment or a remote environment,
+ * depending on the context where your program is executing.
+ */
 package object scala {
   // We have this here so that we always have generated TypeInformationS when
   // using the Scala API


[59/60] git commit: [scala] Add and* methods to AggregateDateSet

Posted by al...@apache.org.
[scala] Add and* methods to AggregateDateSet

And create AggregateDataSet in the first place, to add the methods to.


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

Branch: refs/heads/master
Commit: c778d289b3894832085af5821f046adc3396215d
Parents: b8a8780
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Sep 20 08:08:03 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 13:43:03 2014 +0200

----------------------------------------------------------------------
 .../flink/api/scala/AggregateDataSet.scala      | 90 ++++++++++++++++++++
 .../org/apache/flink/api/scala/DataSet.scala    | 19 +----
 .../apache/flink/api/scala/GroupedDataSet.scala | 47 ++++------
 .../api/scala/ScalaAPICompletenessTest.scala    |  3 +
 .../scala/operators/AggregateOperatorTest.scala |  4 +-
 .../translation/AggregateTranslationTest.scala  |  2 +-
 6 files changed, 118 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c778d289/flink-scala/src/main/scala/org/apache/flink/api/scala/AggregateDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/AggregateDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/AggregateDataSet.scala
new file mode 100644
index 0000000..aac7fbb
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/AggregateDataSet.scala
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala
+
+import org.apache.flink.api.java.aggregation.Aggregations
+import org.apache.flink.api.scala.operators.ScalaAggregateOperator
+
+import scala.reflect.ClassTag
+
+class AggregateDataSet[T: ClassTag](set: ScalaAggregateOperator[T])
+  extends DataSet[T](set) {
+
+  /**
+   * Adds the given aggregation on the given field to the previous aggregation operation.
+   *
+   * This only works on Tuple DataSets.
+   */
+  def and(agg: Aggregations, field: Int): AggregateDataSet[T] = {
+    set.and(agg, field)
+    this
+  }
+
+  /**
+   * Adds the given aggregation on the given field to the previous aggregation operation.
+   *
+   * This only works on CaseClass DataSets.
+   */
+  def and(agg: Aggregations, field: String): AggregateDataSet[T] = {
+    val fieldIndex = fieldNames2Indices(set.getType, Array(field))(0)
+    set.and(agg, fieldIndex)
+    this
+  }
+
+  /**
+   * Syntactic sugar for [[and]] with `SUM`
+   */
+  def andSum(field: Int) = {
+    and(Aggregations.SUM, field)
+  }
+
+  /**
+   * Syntactic sugar for [[and]] with `MAX`
+   */
+  def andMax(field: Int) = {
+    and(Aggregations.MAX, field)
+  }
+
+  /**
+   * Syntactic sugar for [[and]] with `MIN`
+   */
+  def andMin(field: Int) = {
+    and(Aggregations.MIN, field)
+  }
+
+  /**
+   * Syntactic sugar for [[and]] with `SUM`
+   */
+  def andSum(field: String) = {
+    and(Aggregations.SUM, field)
+  }
+
+  /**
+   * Syntactic sugar for [[and]] with `MAX`
+   */
+  def andMax(field: String) = {
+    and(Aggregations.MAX, field)
+  }
+
+  /**
+   * Syntactic sugar for [[and]] with `MIN`
+   */
+  def andMin(field: String) = {
+    and(Aggregations.MIN, field)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c778d289/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 8aabce4..4aa432f 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -29,7 +29,6 @@ import org.apache.flink.api.java.operators.Keys.FieldPositionKeys
 import org.apache.flink.api.java.operators._
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
 import org.apache.flink.api.scala.operators.{ScalaCsvOutputFormat, ScalaAggregateOperator}
-import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.core.fs.{FileSystem, Path}
 import org.apache.flink.types.TypeInformation
@@ -367,12 +366,8 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    *
    * This only works on Tuple DataSets.
    */
-  def aggregate(agg: Aggregations, field: Int): DataSet[T] = set match {
-    case aggregation: ScalaAggregateOperator[T] =>
-      aggregation.and(agg, field)
-      wrap(aggregation)
-
-    case _ => wrap(new ScalaAggregateOperator[T](set, agg, field))
+  def aggregate(agg: Aggregations, field: Int): AggregateDataSet[T] = {
+    new AggregateDataSet(new ScalaAggregateOperator[T](set, agg, field))
   }
 
   /**
@@ -382,16 +377,10 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    *
    * This only works on CaseClass DataSets.
    */
-  def aggregate(agg: Aggregations, field: String): DataSet[T] = {
+  def aggregate(agg: Aggregations, field: String): AggregateDataSet[T] = {
     val fieldIndex = fieldNames2Indices(set.getType, Array(field))(0)
 
-    set match {
-      case aggregation: ScalaAggregateOperator[T] =>
-        aggregation.and(agg, fieldIndex)
-        wrap(aggregation)
-
-      case _ => wrap(new ScalaAggregateOperator[T](set, agg, fieldIndex))
-    }
+    new AggregateDataSet(new ScalaAggregateOperator[T](set, agg, fieldIndex))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c778d289/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
index 88a8c7c..0f2370d 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
@@ -19,7 +19,6 @@ package org.apache.flink.api.scala
 
 import org.apache.flink.api.common.InvalidProgramException
 import org.apache.flink.api.scala.operators.ScalaAggregateOperator
-import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 
 import scala.collection.JavaConverters._
 
@@ -67,7 +66,7 @@ trait GroupedDataSet[T] {
    *
    * This only works on Tuple DataSets.
    */
-  def aggregate(agg: Aggregations, field: Int): DataSet[T]
+  def aggregate(agg: Aggregations, field: Int): AggregateDataSet[T]
 
   /**
    * Creates a new [[DataSet]] by aggregating the specified field using the given aggregation
@@ -76,37 +75,37 @@ trait GroupedDataSet[T] {
    *
    * This only works on CaseClass DataSets.
    */
-  def aggregate(agg: Aggregations, field: String): DataSet[T]
+  def aggregate(agg: Aggregations, field: String): AggregateDataSet[T]
 
   /**
    * Syntactic sugar for [[aggregate]] with `SUM`
    */
-  def sum(field: Int): DataSet[T]
+  def sum(field: Int): AggregateDataSet[T]
 
   /**
    * Syntactic sugar for [[aggregate]] with `MAX`
    */
-  def max(field: Int): DataSet[T]
+  def max(field: Int): AggregateDataSet[T]
 
   /**
    * Syntactic sugar for [[aggregate]] with `MIN`
    */
-  def min(field: Int): DataSet[T]
+  def min(field: Int): AggregateDataSet[T]
 
   /**
    * Syntactic sugar for [[aggregate]] with `SUM`
    */
-  def sum(field: String): DataSet[T]
+  def sum(field: String): AggregateDataSet[T]
 
   /**
    * Syntactic sugar for [[aggregate]] with `MAX`
    */
-  def max(field: String): DataSet[T]
+  def max(field: String): AggregateDataSet[T]
 
   /**
    * Syntactic sugar for [[aggregate]] with `MIN`
    */
-  def min(field: String): DataSet[T]
+  def min(field: String): AggregateDataSet[T]
 
   /**
    * Creates a new [[DataSet]] by merging the elements of each group (elements with the same key)
@@ -194,47 +193,37 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
   /** Convenience methods for creating the [[UnsortedGrouping]] */
   private def createUnsortedGrouping(): Grouping[T] = new UnsortedGrouping[T](set, keys)
 
-  def aggregate(agg: Aggregations, field: String): DataSet[T] = {
+  def aggregate(agg: Aggregations, field: String): AggregateDataSet[T] = {
     val fieldIndex = fieldNames2Indices(set.getType, Array(field))(0)
 
-    set match {
-      case aggregation: ScalaAggregateOperator[T] =>
-        aggregation.and(agg, fieldIndex)
-        wrap(aggregation)
-
-      case _ => wrap(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, fieldIndex))
-    }
+    new AggregateDataSet(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, fieldIndex))
   }
 
-  def aggregate(agg: Aggregations, field: Int): DataSet[T] = set match {
-    case aggregation: ScalaAggregateOperator[T] =>
-      aggregation.and(agg, field)
-      wrap(aggregation)
-
-    case _ => wrap(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, field))
+  def aggregate(agg: Aggregations, field: Int): AggregateDataSet[T] = {
+    new AggregateDataSet(new ScalaAggregateOperator[T](createUnsortedGrouping(), agg, field))
   }
 
-  def sum(field: Int): DataSet[T] = {
+  def sum(field: Int) = {
     aggregate(Aggregations.SUM, field)
   }
 
-  def max(field: Int): DataSet[T] = {
+  def max(field: Int) = {
     aggregate(Aggregations.MAX, field)
   }
 
-  def min(field: Int): DataSet[T] = {
+  def min(field: Int) = {
     aggregate(Aggregations.MIN, field)
   }
 
-  def sum(field: String): DataSet[T] = {
+  def sum(field: String) = {
     aggregate(Aggregations.SUM, field)
   }
 
-  def max(field: String): DataSet[T] = {
+  def max(field: String) = {
     aggregate(Aggregations.MAX, field)
   }
 
-  def min(field: String): DataSet[T] = {
+  def min(field: String) = {
     aggregate(Aggregations.MIN, field)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c778d289/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
index 97939f0..6e84271 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/ScalaAPICompletenessTest.scala
@@ -152,6 +152,9 @@ class ScalaAPICompletenessTest {
     checkMethods("SortedGrouping", "GroupedDataSet",
       classOf[SortedGrouping[_]], classOf[GroupedDataSet[_]])
 
+    checkMethods("AggregateOperator", "AggregateDataSet",
+      classOf[AggregateOperator[_]], classOf[AggregateDataSet[_]])
+
     checkMethods("SingleInputOperator", "DataSet",
       classOf[SingleInputOperator[_, _, _]], classOf[DataSet[_]])
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c778d289/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
index ea35138..4dbc5fc 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
@@ -108,10 +108,10 @@ class AggregateOperatorTest {
       val tupleDs = env.fromCollection(emptyTupleData)
 
       // should work: multiple aggregates
-      tupleDs.aggregate(Aggregations.SUM, 0).aggregate(Aggregations.MIN, 4)
+      tupleDs.aggregate(Aggregations.SUM, 0).and(Aggregations.MIN, 4)
 
       // should work: nested aggregates
-      tupleDs.aggregate(Aggregations.MIN, 2).aggregate(Aggregations.SUM, 1)
+      tupleDs.aggregate(Aggregations.MIN, 2).and(Aggregations.SUM, 1)
 
       // should not work: average on string
       try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c778d289/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
index 7501f82..2ca0c01 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
@@ -35,7 +35,7 @@ class AggregateTranslationTest {
 
       val initialData = env.fromElements((3.141592, "foobar", 77L))
 
-      initialData.groupBy(0).aggregate(Aggregations.MIN, 1).aggregate(Aggregations.SUM, 2).print()
+      initialData.groupBy(0).aggregate(Aggregations.MIN, 1).and(Aggregations.SUM, 2).print()
 
       val p: Plan = env.createProgramPlan()
       val sink = p.getDataSinks.iterator.next


[42/60] git commit: Fix rename of java example package to examples

Posted by al...@apache.org.
Fix rename of java example package to examples

Some occurences in comments and POMs where not updated.

Also change signature of join and coGroup to always return a value, not
an Option.


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

Branch: refs/heads/master
Commit: a41a29b4ec14d385bbb297bc568d294e99a6ab82
Parents: 4cce46e
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sun Sep 14 19:32:45 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 flink-examples/flink-java-examples/pom.xml      | 20 ++++++++++----------
 .../examples/scala/clustering/KMeans.scala      |  3 ++-
 .../scala/graph/ConnectedComponents.scala       |  8 +++++---
 .../scala/graph/EnumTrianglesBasic.scala        |  4 ++--
 .../examples/scala/graph/EnumTrianglesOpt.scala |  4 ++--
 .../examples/scala/graph/PageRankBasic.scala    |  6 +++---
 .../scala/graph/TransitiveClosureNaive.scala    |  2 +-
 .../scala/relational/WebLogAnalysis.scala       |  6 +++---
 .../examples/scala/wordcount/WordCount.scala    |  2 +-
 .../apache/flink/api/scala/coGroupDataSet.scala |  9 ++++-----
 .../apache/flink/api/scala/joinDataSet.scala    |  9 ++++-----
 .../DeltaIterationTranslationTest.scala         |  2 +-
 .../WordCountPOJOITCase.java                    |  2 +-
 13 files changed, 39 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-java-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml
index 3169829..836b0bc 100644
--- a/flink-examples/flink-java-examples/pom.xml
+++ b/flink-examples/flink-java-examples/pom.xml
@@ -61,7 +61,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.clustering.KMeans</program-class>
+									<program-class>org.apache.flink.examples.java.clustering.KMeans</program-class>
 								</manifestEntries>
 							</archive>
 
@@ -86,7 +86,7 @@ under the License.
 				
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.graph.TransitiveClosureNaive</program-class>
+									<program-class>org.apache.flink.examples.java.graph.TransitiveClosureNaive</program-class>
 								</manifestEntries>
 							</archive>
 				
@@ -110,7 +110,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.graph.ConnectedComponents</program-class>
+									<program-class>org.apache.flink.examples.java.graph.ConnectedComponents</program-class>
 								</manifestEntries>
 							</archive>
 
@@ -134,7 +134,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.graph.EnumTrianglesBasic</program-class>
+									<program-class>org.apache.flink.examples.java.graph.EnumTrianglesBasic</program-class>
 								</manifestEntries>
 							</archive>
 
@@ -160,7 +160,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.graph.EnumTrianglesOpt</program-class>
+									<program-class>org.apache.flink.examples.java.graph.EnumTrianglesOpt</program-class>
 								</manifestEntries>
 							</archive>
 
@@ -186,7 +186,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.graph.PageRankBasic</program-class>
+									<program-class>org.apache.flink.examples.java.graph.PageRankBasic</program-class>
 								</manifestEntries>
 							</archive>
 
@@ -214,7 +214,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.relational.TPCHQuery10</program-class>
+									<program-class>org.apache.flink.examples.java.relational.TPCHQuery10</program-class>
 								</manifestEntries>
 							</archive>
 							<includes>
@@ -237,7 +237,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.relational.TPCHQuery3</program-class>
+									<program-class>org.apache.flink.examples.java.relational.TPCHQuery3</program-class>
 								</manifestEntries>
 							</archive>
 							<includes>
@@ -259,7 +259,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.relational.WebLogAnalysis</program-class>
+									<program-class>org.apache.flink.examples.java.relational.WebLogAnalysis</program-class>
 								</manifestEntries>
 							</archive>
 
@@ -284,7 +284,7 @@ under the License.
 
 							<archive>
 								<manifestEntries>
-									<program-class>org.apache.flink.example.java.wordcount.WordCount</program-class>
+									<program-class>org.apache.flink.examples.java.wordcount.WordCount</program-class>
 								</manifestEntries>
 							</archive>
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
index a0bb874..e3b499e 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/clustering/KMeans.scala
@@ -58,7 +58,8 @@ import scala.collection.JavaConverters._
  * {{{
  *   KMeans <points path> <centers path> <result path> <num iterations>
  * }}}
- * If no parameters are provided, the program is run with default data from `KMeansData`
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.examples.java.clustering.util.KMeansData]]
  * and 10 iterations.
  *
  * This example shows how to use:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
index 8e3fbfd..df28ad0 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/ConnectedComponents.scala
@@ -19,6 +19,7 @@ package org.apache.flink.examples.scala.graph
 
 import org.apache.flink.api.scala._
 import org.apache.flink.examples.java.graph.util.ConnectedComponentsData
+import org.apache.flink.util.Collector
 
 /**
  * An implementation of the connected components algorithm, using a delta iteration.
@@ -49,7 +50,7 @@ import org.apache.flink.examples.java.graph.util.ConnectedComponentsData
  * }}}
  *   
  * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.example.java.graph.util.ConnectedComponentsData]] and 10 iterations.
+ * [[org.apache.flink.examples.java.graph.util.ConnectedComponentsData]] and 10 iterations.
  * 
  *
  * This example shows how to use:
@@ -79,7 +80,7 @@ object ConnectedComponents {
 
         // apply the step logic: join with the edges
         val allNeighbors = ws.join(edges).where(0).equalTo(0) { (vertex, edge) =>
-          Some((edge._2, vertex._2))
+          (edge._2, vertex._2)
         }
 
         // select the minimum neighbor
@@ -87,7 +88,8 @@ object ConnectedComponents {
 
         // update if the component of the candidate is smaller
         val updatedComponents = minNeighbors.join(s).where(0).equalTo(0) {
-          (newVertex, oldVertex) => if (newVertex._2 < oldVertex._2) Some(newVertex) else None
+          (newVertex, oldVertex, out: Collector[(Long, Long)]) =>
+            if (newVertex._2 < oldVertex._2) out.collect(newVertex)
         }
 
         // delta and new workset are identical

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
index 0ad2e81..c920c31 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
@@ -57,7 +57,7 @@ import scala.collection.mutable.MutableList
  * }}}
  * <br>
  * If no parameters are provided, the program is run with default data from 
- * [[org.apache.flink.example.java.graph.util.EnumTrianglesData]]
+ * [[org.apache.flink.examples.java.graph.util.EnumTrianglesData]]
  * 
  * <p>
  * This example shows how to use:
@@ -87,7 +87,7 @@ object EnumTrianglesBasic {
 						// build triads
 						.groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
 						// filter triads
-						.join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => Some(t) }
+						.join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => t }
 		
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
index 24b7978..80cce35 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
@@ -61,7 +61,7 @@ import scala.collection.mutable.MutableList
  * }}}
  *
  * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.example.java.graph.util.EnumTrianglesData]].
+ * [[org.apache.flink.examples.java.graph.util.EnumTrianglesData]].
  *
  * This example shows how to use:
  *
@@ -108,7 +108,7 @@ object EnumTrianglesOpt {
       // build triads
       .groupBy("v1").sortGroup("v2", Order.ASCENDING).reduceGroup(new TriadBuilder())
       // filter triads
-      .join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => Some(t)}
+      .join(edgesById).where("v2", "v3").equalTo("v1", "v2") { (t, _) => t}
 
     // emit result
     if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
index cdd04f7..5f515c4 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/PageRankBasic.scala
@@ -52,7 +52,7 @@ import org.apache.flink.util.Collector
  * }}}
  *
  * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.example.java.graph.util.PageRankData]] and 10 iterations.
+ * [[org.apache.flink.examples.java.graph.util.PageRankData]] and 10 iterations.
  * 
  * This example shows how to use:
  *
@@ -108,9 +108,9 @@ object PageRankBasic {
 
         // terminate if no rank update was significant
         val termination = currentRanks.join(newRanks).where("pageId").equalTo("pageId") {
-          (current, next) =>
+          (current, next, out: Collector[Int]) =>
             // check for significant update
-            if (math.abs(current.rank - next.rank) > EPSILON) Some(1) else None
+            if (math.abs(current.rank - next.rank) > EPSILON) out.collect(1)
         }
 
         (newRanks, termination)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
index 4c79a1b..25347ca 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
@@ -37,7 +37,7 @@ object TransitiveClosureNaive {
 			val nextPaths = prevPaths
 				.join(edges)
 				.where(1).equalTo(0) {
-					(left, right) => Some((left._1,right._2))
+					(left, right) => (left._1,right._2)
 				}
 				.union(prevPaths)
 				.groupBy(0, 1)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
index 156d635..100b3d2 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/WebLogAnalysis.scala
@@ -45,7 +45,7 @@ import org.apache.flink.util.Collector
  *
  * Input files are plain text CSV files using the pipe character ('|') as field separator.
  * The tables referenced in the query can be generated using the
- * [org.apache.flink.example.java.relational.util.WebLogDataGenerator]] and
+ * [org.apache.flink.examples.java.relational.util.WebLogDataGenerator]] and
  * have the following schemas
  *
  * {{{
@@ -77,7 +77,7 @@ import org.apache.flink.util.Collector
  * }}}
  *
  * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.example.java.relational.util.WebLogData]].
+ * [[org.apache.flink.examples.java.relational.util.WebLogData]].
  *
  * This example shows how to use:
  *
@@ -109,7 +109,7 @@ object WebLogAnalysis {
       .filter(visit => visit._2.substring(0, 4).toInt == 2007)
 
     val joinDocsRanks = filteredDocs.join(filteredRanks).where(0).equalTo(1) {
-      (doc, rank) => Some(rank)
+      (doc, rank) => rank
     }
 
     val result = joinDocsRanks.coGroup(filteredVisits).where(1).equalTo(0) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
index 9d93ca8..989ed87 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala
@@ -32,7 +32,7 @@ import org.apache.flink.examples.java.wordcount.util.WordCountData
  * }}}
  *
  * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.example.java.wordcount.util.WordCountData]]
+ * [[org.apache.flink.examples.java.wordcount.util.WordCountData]]
  *
  * This example shows how to:
  *

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
index f936b43..b0661a1 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/coGroupDataSet.scala
@@ -63,11 +63,10 @@ trait CoGroupDataSet[T, O] extends DataSet[(Array[T], Array[O])] {
 
   /**
    * Creates a new [[DataSet]] where the result for each pair of co-grouped element lists is the
-   * result of the given function. You can either return an element or choose to return [[None]],
-   * which allows implementing a filter directly in the coGroup function.
+   * result of the given function.
    */
   def apply[R: TypeInformation: ClassTag](
-      fun: (TraversableOnce[T], TraversableOnce[O]) => Option[R]): DataSet[R]
+      fun: (TraversableOnce[T], TraversableOnce[O]) => R): DataSet[R]
 
   /**
    * Creates a new [[DataSet]] where the result for each pair of co-grouped element lists is the
@@ -100,11 +99,11 @@ private[flink] class CoGroupDataSetImpl[T, O](
     otherKeys: Keys[O]) extends DataSet(coGroupOperator) with CoGroupDataSet[T, O] {
 
   def apply[R: TypeInformation: ClassTag](
-      fun: (TraversableOnce[T], TraversableOnce[O]) => Option[R]): DataSet[R] = {
+      fun: (TraversableOnce[T], TraversableOnce[O]) => R): DataSet[R] = {
     Validate.notNull(fun, "CoGroup function must not be null.")
     val coGrouper = new CoGroupFunction[T, O, R] {
       def coGroup(left: java.lang.Iterable[T], right: java.lang.Iterable[O], out: Collector[R]) = {
-        fun(left.iterator.asScala, right.iterator.asScala) map { out.collect(_) }
+        out.collect(fun(left.iterator.asScala, right.iterator.asScala))
       }
     }
     val coGroupOperator = new CoGroupOperator[T, O, R](thisSet.set, otherSet.set, thisKeys,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
index 3f9f6e9..c3f4dda 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala
@@ -62,10 +62,9 @@ trait JoinDataSet[T, O] extends DataSet[(T, O)] {
 
   /**
    * Creates a new [[DataSet]] where the result for each pair of joined elements is the result
-   * of the given function. You can either return an element or choose to return [[None]],
-   * which allows implementing a filter directly in the join function.
+   * of the given function.
    */
-  def apply[R: TypeInformation: ClassTag](fun: (T, O) => Option[R]): DataSet[R]
+  def apply[R: TypeInformation: ClassTag](fun: (T, O) => R): DataSet[R]
 
   /**
    * Creates a new [[DataSet]] by passing each pair of joined values to the given function.
@@ -107,11 +106,11 @@ private[flink] class JoinDataSetImpl[T, O](
   extends DataSet(joinOperator)
   with JoinDataSet[T, O] {
 
-  def apply[R: TypeInformation: ClassTag](fun: (T, O) => Option[R]): DataSet[R] = {
+  def apply[R: TypeInformation: ClassTag](fun: (T, O) => R): DataSet[R] = {
     Validate.notNull(fun, "Join function must not be null.")
     val joiner = new FlatJoinFunction[T, O, R] {
       def join(left: T, right: O, out: Collector[R]) = {
-        fun(left, right) map { out.collect(_) }
+        out.collect(fun(left, right))
       }
     }
     val joinOperator = new EquiJoin[T, O, R](thisSet, otherSet, thisKeys,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
index 5631cbb..9b953ee 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
@@ -58,7 +58,7 @@ class DeltaIterationTranslationTest {
       val result = initialSolutionSet.iterateDelta(initialWorkSet, NUM_ITERATIONS, ITERATION_KEYS) {
         (s, ws) =>
           val wsSelfJoin = ws.map(new IdentityMapper[(Double, String)]())
-            .join(ws).where(1).equalTo(1) { (l, r) => Some(l) }
+            .join(ws).where(1).equalTo(1) { (l, r) => l }
 
           val joined = wsSelfJoin.join(s).where(1).equalTo(2).apply(new SolutionWorksetJoin)
           (joined, joined.map(new NextWorksetMapper).name(BEFORE_NEXT_WORKSET_MAP))

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a41a29b4/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountPOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountPOJOITCase.java
index 5639ed7..0928a46 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountPOJOITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountPOJOITCase.java
@@ -18,7 +18,7 @@
 //
 //package org.apache.flink.test.exampleJavaPrograms;
 //
-////import org.apache.flink.example.java.wordcount.WordCountPOJO;
+////import org.apache.flink.examples.java.wordcount.WordCountPOJO;
 //import org.apache.flink.test.testdata.WordCountData;
 //import org.apache.flink.test.util.JavaProgramTestBase;
 //import org.junit.Ignore;


[02/60] Rewrite the Scala API as (somewhat) thin Layer on Java API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala
new file mode 100644
index 0000000..1535eed
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.io
+
+import org.apache.flink.api.java.io.CollectionInputFormat
+import org.junit.Assert.assertEquals
+import org.junit.Assert.assertNotNull
+import org.junit.Assert.assertTrue
+import org.apache.flink.api.java.typeutils.BasicTypeInfo
+import org.apache.flink.core.io.GenericInputSplit
+import org.junit.Test
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+import java.io.ObjectInputStream
+import java.io.ObjectOutputStream
+import org.apache.flink.api.scala._
+import scala.collection.JavaConverters._
+
+class ElementType(val id: Int) {
+  def this() {
+    this(-1)
+  }
+
+  override def equals(obj: Any): Boolean = {
+    if (obj != null && obj.isInstanceOf[ElementType]) {
+      val et = obj.asInstanceOf[ElementType]
+      et.id == this.id
+    }
+    else {
+      false
+    }
+  }
+}
+
+class CollectionInputFormatTest {
+
+  @Test
+  def testSerializability(): Unit = {
+
+    val inputCollection = Seq(new ElementType(1), new ElementType(2), new ElementType(3))
+    val info = createTypeInformation[ElementType]
+
+    val inputFormat: CollectionInputFormat[ElementType] = new
+        CollectionInputFormat[ElementType](inputCollection.asJava, info.createSerializer())
+
+    val buffer = new ByteArrayOutputStream
+    val out = new ObjectOutputStream(buffer)
+
+    out.writeObject(inputFormat)
+
+    val in = new ObjectInputStream(new ByteArrayInputStream(buffer.toByteArray))
+    val serializationResult: AnyRef = in.readObject
+
+    assertNotNull(serializationResult)
+    assertTrue(serializationResult.isInstanceOf[CollectionInputFormat[_]])
+
+    val result = serializationResult.asInstanceOf[CollectionInputFormat[ElementType]]
+    val inputSplit = new GenericInputSplit
+    inputFormat.open(inputSplit)
+    result.open(inputSplit)
+
+    while (!inputFormat.reachedEnd && !result.reachedEnd) {
+      val expectedElement = inputFormat.nextRecord(null)
+      val actualElement = result.nextRecord(null)
+      assertEquals(expectedElement, actualElement)
+    }
+  }
+
+  @Test
+  def testSerializabilityStrings(): Unit = {
+    val data = Seq("To bey or not to be,--that is the question:--",
+      "Whether 'tis nobler in the mind to suffer", "The slings and arrows of outrageous fortune",
+      "Or to take arms against a sea of troubles,", "And by opposing end them?--To die," +
+        "--to sleep,--", "No more; and by a sleep to say we end", "The heartache, " +
+        "and the thousand natural shocks", "That flesh is heir to,--'tis a consummation",
+      "Devoutly to be wish'd. To die,--to sleep;--", "To sleep! perchance to dream:--ay, " +
+        "there's the rub;", "For in that sleep of death what dreams may come,",
+      "When we have shuffled off this mortal coil,", "Must give us pause: there's the respect",
+      "That makes calamity of so long life;", "For who would bear the whips and scorns of time,",
+      "The oppressor's wrong, the proud man's contumely,", "The pangs of despis'd love, " +
+        "the law's delay,", "The insolence of office, and the spurns",
+      "That patient merit of the unworthy takes,", "When he himself might his quietus make",
+      "With a bare bodkin? who would these fardels bear,", "To grunt and sweat under a weary " +
+        "life,", "But that the dread of something after death,--", "The undiscover'd country, " +
+        "from whose bourn", "No traveller returns,--puzzles the will,",
+      "And makes us rather bear those ills we have", "Than fly to others that we know not of?",
+      "Thus conscience does make cowards of us all;", "And thus the native hue of resolution",
+      "Is sicklied o'er with the pale cast of thought;", "And enterprises of great pith and " +
+        "moment,", "With this regard, their currents turn awry,", "And lose the name of action" +
+        ".--Soft you now!", "The fair Ophelia!--Nymph, in thy orisons",
+      "Be all my sins remember'd.")
+
+    val inputFormat = new CollectionInputFormat[String](
+      data.asJava,
+      BasicTypeInfo.STRING_TYPE_INFO.createSerializer)
+    val baos = new ByteArrayOutputStream
+    val oos = new ObjectOutputStream(baos)
+
+    oos.writeObject(inputFormat)
+    oos.close()
+
+    val bais = new ByteArrayInputStream(baos.toByteArray)
+    val ois = new ObjectInputStream(bais)
+    val result: AnyRef = ois.readObject
+
+    assertTrue(result.isInstanceOf[CollectionInputFormat[_]])
+    var i: Int = 0
+    val in = result.asInstanceOf[CollectionInputFormat[String]]
+    in.open(new GenericInputSplit)
+
+    while (!in.reachedEnd) {
+      assertEquals(data(i), in.nextRecord(""))
+      i += 1
+    }
+    assertEquals(data.length, i)
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CsvInputFormatTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CsvInputFormatTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CsvInputFormatTest.scala
new file mode 100644
index 0000000..e85b259
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/io/CsvInputFormatTest.scala
@@ -0,0 +1,288 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.io
+
+import org.apache.flink.api.scala.operators.ScalaCsvInputFormat
+import org.junit.Assert.assertEquals
+import org.junit.Assert.assertNotNull
+import org.junit.Assert.assertNull
+import org.junit.Assert.assertTrue
+import org.junit.Assert.fail
+import java.io.File
+import java.io.FileOutputStream
+import java.io.FileWriter
+import java.io.OutputStreamWriter
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.core.fs.FileInputSplit
+import org.apache.flink.core.fs.Path
+import org.junit.Test
+import org.apache.flink.api.scala._
+
+class CsvInputFormatTest {
+
+  private final val PATH: Path = new Path("an/ignored/file/")
+  private final val FIRST_PART: String = "That is the first part"
+  private final val SECOND_PART: String = "That is the second part"
+
+  @Test
+  def readStringFields():Unit = {
+    try {
+      val fileContent = "abc|def|ghijk\nabc||hhg\n|||"
+      val split = createTempFile(fileContent)
+      val format = new ScalaCsvInputFormat[(String, String, String)](
+        PATH, createTypeInformation[(String, String, String)])
+      format.setDelimiter("\n")
+      format.setFieldDelimiter('|')
+      val parameters = new Configuration
+      format.configure(parameters)
+      format.open(split)
+      var result: (String, String, String) = null
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals("abc", result._1)
+      assertEquals("def", result._2)
+      assertEquals("ghijk", result._3)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals("abc", result._1)
+      assertEquals("", result._2)
+      assertEquals("hhg", result._3)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals("", result._1)
+      assertEquals("", result._2)
+      assertEquals("", result._3)
+      result = format.nextRecord(result)
+      assertNull(result)
+      assertTrue(format.reachedEnd)
+    }
+    catch {
+      case ex: Exception => {
+        ex.printStackTrace()
+        fail("Test failed due to a " + ex.getClass.getName + ": " + ex.getMessage)
+      }
+    }
+  }
+
+  @Test
+  def readStringFieldsWithTrailingDelimiters(): Unit = {
+    try {
+      val fileContent = "abc|def|ghijk\nabc||hhg\n|||\n"
+      val split = createTempFile(fileContent)
+      val format = new ScalaCsvInputFormat[(String, String, String)](
+        PATH, createTypeInformation[(String, String, String)])
+      format.setDelimiter("\n")
+      format.setFieldDelimiter('|')
+      val parameters = new Configuration
+      format.configure(parameters)
+      format.open(split)
+      var result: (String, String, String) = null
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals("abc", result._1)
+      assertEquals("def", result._2)
+      assertEquals("ghijk", result._3)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals("abc", result._1)
+      assertEquals("", result._2)
+      assertEquals("hhg", result._3)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals("", result._1)
+      assertEquals("", result._2)
+      assertEquals("", result._3)
+      result = format.nextRecord(result)
+      assertNull(result)
+      assertTrue(format.reachedEnd)
+    }
+    catch {
+      case ex: Exception =>
+        ex.printStackTrace()
+        fail("Test failed due to a " + ex.getClass.getName + ": " + ex.getMessage)
+    }
+  }
+
+  @Test
+  def testIntegerFields(): Unit = {
+    try {
+      val fileContent = "111|222|333|444|555\n666|777|888|999|000|\n"
+      val split = createTempFile(fileContent)
+      val format = new ScalaCsvInputFormat[(Int, Int, Int, Int, Int)](
+        PATH, createTypeInformation[(Int, Int, Int, Int, Int)])
+      format.setFieldDelimiter('|')
+      format.configure(new Configuration)
+      format.open(split)
+      var result: (Int, Int, Int, Int, Int) = null
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals(Integer.valueOf(111), result._1)
+      assertEquals(Integer.valueOf(222), result._2)
+      assertEquals(Integer.valueOf(333), result._3)
+      assertEquals(Integer.valueOf(444), result._4)
+      assertEquals(Integer.valueOf(555), result._5)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals(Integer.valueOf(666), result._1)
+      assertEquals(Integer.valueOf(777), result._2)
+      assertEquals(Integer.valueOf(888), result._3)
+      assertEquals(Integer.valueOf(999), result._4)
+      assertEquals(Integer.valueOf(000), result._5)
+      result = format.nextRecord(result)
+      assertNull(result)
+      assertTrue(format.reachedEnd)
+    }
+    catch {
+      case ex: Exception =>
+        fail("Test failed due to a " + ex.getClass.getName + ": " + ex.getMessage)
+    }
+  }
+
+  @Test
+  def testReadFirstN(): Unit = {
+    try {
+      val fileContent = "111|222|333|444|555|\n666|777|888|999|000|\n"
+      val split = createTempFile(fileContent)
+      val format = new ScalaCsvInputFormat[(Int, Int)](PATH, createTypeInformation[(Int, Int)])
+      format.setFieldDelimiter('|')
+      format.configure(new Configuration)
+      format.open(split)
+      var result: (Int, Int) = null
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals(Integer.valueOf(111), result._1)
+      assertEquals(Integer.valueOf(222), result._2)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals(Integer.valueOf(666), result._1)
+      assertEquals(Integer.valueOf(777), result._2)
+      result = format.nextRecord(result)
+      assertNull(result)
+      assertTrue(format.reachedEnd)
+    }
+    catch {
+      case ex: Exception =>
+        fail("Test failed due to a " + ex.getClass.getName + ": " + ex.getMessage)
+    }
+  }
+
+  @Test
+  def testReadSparseWithPositionSetter(): Unit = {
+    try {
+      val fileContent: String = "111|222|333|444|555|666|777|888|999|000|\n000|999|888|777|666" +
+        "|555|444|333|222|111|"
+      val split = createTempFile(fileContent)
+      val format = new ScalaCsvInputFormat[(Int, Int, Int)](
+        PATH,
+        createTypeInformation[(Int, Int, Int)])
+      format.setFieldDelimiter('|')
+      format.setFields(Array(0, 3, 7), Array(classOf[Integer], classOf[Integer], classOf[Integer]))
+      format.configure(new Configuration)
+      format.open(split)
+      var result: (Int, Int, Int) = null
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals(Integer.valueOf(111), result._1)
+      assertEquals(Integer.valueOf(444), result._2)
+      assertEquals(Integer.valueOf(888), result._3)
+      result = format.nextRecord(result)
+      assertNotNull(result)
+      assertEquals(Integer.valueOf(000), result._1)
+      assertEquals(Integer.valueOf(777), result._2)
+      assertEquals(Integer.valueOf(333), result._3)
+      result = format.nextRecord(result)
+      assertNull(result)
+      assertTrue(format.reachedEnd)
+    }
+    catch {
+      case ex: Exception =>
+        fail("Test failed due to a " + ex.getClass.getName + ": " + ex.getMessage)
+    }
+  }
+
+  @Test
+  def testReadSparseWithShuffledPositions(): Unit = {
+    try {
+      val format = new ScalaCsvInputFormat[(Int, Int, Int)](
+        PATH,
+        createTypeInformation[(Int, Int, Int)])
+      format.setFieldDelimiter('|')
+      try {
+        format.setFields(Array(8, 1, 3), Array(classOf[Integer],classOf[Integer],classOf[Integer]))
+        fail("Input sequence should have been rejected.")
+      }
+      catch {
+        case e: IllegalArgumentException => // ignore
+      }
+    }
+    catch {
+      case ex: Exception =>
+        fail("Test failed due to a " + ex.getClass.getName + ": " + ex.getMessage)
+    }
+  }
+
+  private def createTempFile(content: String): FileInputSplit = {
+    val tempFile = File.createTempFile("test_contents", "tmp")
+    tempFile.deleteOnExit()
+    val wrt = new FileWriter(tempFile)
+    wrt.write(content)
+    wrt.close()
+    new FileInputSplit(0, new Path(tempFile.toURI.toString), 0,
+      tempFile.length,Array[String]("localhost"))
+  }
+
+  @Test
+  def testWindowsLineEndRemoval(): Unit = {
+    this.testRemovingTrailingCR("\n", "\n")
+    this.testRemovingTrailingCR("\r\n", "\r\n")
+    this.testRemovingTrailingCR("\r\n", "\n")
+  }
+
+  private def testRemovingTrailingCR(lineBreakerInFile: String, lineBreakerSetup: String) {
+    var tempFile: File = null
+    val fileContent = FIRST_PART + lineBreakerInFile + SECOND_PART + lineBreakerInFile
+    try {
+      tempFile = File.createTempFile("CsvInputFormatTest", "tmp")
+      tempFile.deleteOnExit()
+      tempFile.setWritable(true)
+      val wrt = new OutputStreamWriter(new FileOutputStream(tempFile))
+      wrt.write(fileContent)
+      wrt.close()
+      val inputFormat = new ScalaCsvInputFormat[Tuple1[String]](new Path(tempFile.toURI.toString),
+        createTypeInformation[Tuple1[String]])
+      val parameters = new Configuration
+      inputFormat.configure(parameters)
+      inputFormat.setDelimiter(lineBreakerSetup)
+      val splits = inputFormat.createInputSplits(1)
+      inputFormat.open(splits(0))
+      var result = inputFormat.nextRecord(null)
+      assertNotNull("Expecting to not return null", result)
+      assertEquals(FIRST_PART, result._1)
+      result = inputFormat.nextRecord(result)
+      assertNotNull("Expecting to not return null", result)
+      assertEquals(SECOND_PART, result._1)
+    }
+    catch {
+      case t: Throwable =>
+        System.err.println("test failed with exception: " + t.getMessage)
+        t.printStackTrace(System.err)
+        fail("Test erroneous")
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
new file mode 100644
index 0000000..ae9fe22
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/AggregateOperatorTest.scala
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators
+
+import org.junit.Assert
+import org.apache.flink.api.common.InvalidProgramException
+import org.apache.flink.api.java.aggregation.Aggregations
+import org.apache.flink.api.java.aggregation.UnsupportedAggregationTypeException
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+class AggregateOperatorTest {
+
+  private final val emptyTupleData = Array[(Int, Long, String, Long, Int)]()
+  private final val tupleTypeInfo = createTypeInformation[(Int, Long, String, Long, Int)]
+  private final val emptyLongData = Array[Long]()
+
+  @Test
+  def testFieldsAggregate(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should work
+    try {
+      tupleDs.aggregate(Aggregations.SUM, 1)
+    } catch {
+      case e: Exception => Assert.fail()
+    }
+
+    // should not work: index out of bounds
+    try {
+      tupleDs.aggregate(Aggregations.SUM, 10)
+      Assert.fail()
+    } catch {
+      case iae: IllegalArgumentException =>
+      case e: Exception => Assert.fail()
+    }
+
+    val longDs = env.fromCollection(emptyLongData)
+
+    // should not work: not applied to tuple DataSet
+    try {
+      longDs.aggregate(Aggregations.MIN, 1)
+      Assert.fail()
+    } catch {
+      case uoe: InvalidProgramException =>
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
+  def testAggregationTypes(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+
+      val tupleDs = env.fromCollection(emptyTupleData)
+
+      tupleDs.aggregate(Aggregations.SUM, 0).aggregate(Aggregations.MIN, 4)
+      tupleDs.aggregate(Aggregations.MIN, 2).aggregate(Aggregations.SUM, 1)
+      try {
+        tupleDs.aggregate(Aggregations.SUM, 2)
+        Assert.fail()
+      } catch {
+        case iae: UnsupportedAggregationTypeException =>
+      }
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        Assert.fail(e.getMessage)
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
new file mode 100644
index 0000000..3608a50
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators
+
+import java.io.Serializable
+import org.apache.flink.api.common.InvalidProgramException
+import org.junit.Assert
+import org.junit.Ignore
+import org.junit.Test
+import org.apache.flink.api.scala._
+
+class CoGroupOperatorTest {
+
+  private val emptyTupleData = Array[(Int, Long, String, Long, Int)]()
+  private var customTypeData = Array[CustomType](new CustomType())
+
+  @Test
+  def testCoGroupKeyFields1(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should work
+    try {
+      ds1.coGroup(ds2).where(0).equalTo(0)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, incompatible key types
+    ds1.coGroup(ds2).where(0).equalTo(2)
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyFields3(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, incompatible number of key fields
+    ds1.coGroup(ds2).where(0, 1).equalTo(2)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testCoGroupKeyFields4(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, field position out of range
+    ds1.coGroup(ds2).where(5).equalTo(0)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testCoGroupKeyFields5(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should not work, negative field position
+    ds1.coGroup(ds2).where(-1).equalTo(-1)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testCoGroupKeyFields6(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should not work, field position key on custom data type
+    ds1.coGroup(ds2).where(5).equalTo(0)
+  }
+
+  @Ignore
+  @Test
+  def testCoGroupKeyExpressions1(): Unit =  {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should work
+    try {
+//      ds1.coGroup(ds2).where("i").equalTo("i");
+
+    }catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Ignore
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyExpressions2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, incompatible key types
+//    ds1.coGroup(ds2).where("i").equalTo("s")
+  }
+
+  @Ignore
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyExpressions3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, incompatible number of keys
+//    ds1.coGroup(ds2).where("i", "s").equalTo("s")
+  }
+
+  @Ignore
+  @Test(expected = classOf[IllegalArgumentException])
+  def testCoGroupKeyExpressions4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+
+    // should not work, key non-existent
+//    ds1.coGroup(ds2).where("myNonExistent").equalTo("i")
+  }
+
+  @Test
+  def testCoGroupKeySelectors1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should work
+    try {
+      ds1.coGroup(ds2).where { _.l } equalTo { _.l }
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
+  def testCoGroupKeyMixing1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // Should work
+    try {
+      ds1.coGroup(ds2).where { _.l}.equalTo(3)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
+  def testCoGroupKeyMixing2(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should work
+    try {
+      ds1.coGroup(ds2).where(3).equalTo { _.l }
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyMixing3(): Unit =  {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should not work, incompatible types
+    ds1.coGroup(ds2).where(2).equalTo { _.l }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testCoGroupKeyMixing4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // Should not work, more than one field position key
+    ds1.coGroup(ds2).where(1, 3).equalTo { _.l }
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CustomType.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CustomType.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CustomType.scala
new file mode 100644
index 0000000..3b82f4c
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/CustomType.scala
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators
+
+import java.io.Serializable
+
+/**
+ * A custom data type that is used by the operator Tests.
+ */
+class CustomType(var i:Int, var l: Long, var s: String) extends Serializable {
+  def this() {
+    this(0, 0, null)
+  }
+
+  override def toString: String = {
+    i + "," + l + "," + s
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
new file mode 100644
index 0000000..693cd87
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/DistinctOperatorTest.scala
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators
+
+import org.junit.Assert
+import org.apache.flink.api.common.InvalidProgramException
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+class DistinctOperatorTest {
+
+  private val emptyTupleData = Array[(Int, Long, String, Long, Int)]()
+  private val customTypeData = Array[CustomType](new CustomType())
+  private val emptyLongData = Array[Long]()
+
+  @Test
+  def testDistinctByKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // Should work
+    try {
+      tupleDs.distinct(0)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testDistinctByKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val longDs = env.fromCollection(emptyLongData)
+
+    // should not work: distinct on basic type
+    longDs.distinct(0)
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testDistinctByKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val customDs = env.fromCollection(customTypeData)
+
+    // should not work: field position key on custom type
+    customDs.distinct(0)
+  }
+
+  @Test
+  def testDistinctByKeyFields4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should work
+    tupleDs.distinct()
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testDistinctByKeyFields5(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val customDs = env.fromCollection(customTypeData)
+
+    // should not work: distinct on custom type
+    customDs.distinct()
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testDistinctByKeyFields6(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should not work, negative field position key
+    tupleDs.distinct(-1)
+  }
+
+  @Test
+  def testDistinctByKeySelector1(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    try {
+      val customDs = env.fromCollection(customTypeData)
+      customDs.distinct {_.l}
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
new file mode 100644
index 0000000..841fd52
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators
+
+import org.junit.Assert
+import org.apache.flink.api.common.InvalidProgramException
+import org.apache.flink.api.common.operators.Order
+import org.junit.Ignore
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+
+class GroupingTest {
+
+  private val emptyTupleData = Array[(Int, Long, String, Long, Int)]()
+  private val customTypeData = Array[CustomType](new CustomType())
+  private val emptyLongData = Array[Long]()
+
+  @Test
+  def testGroupByKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should work
+    try {
+      tupleDs.groupBy(0)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testGroupByKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val longDs = env.fromCollection(emptyLongData)
+
+    // should not work, grouping on basic type
+    longDs.groupBy(0)
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testGroupByKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val customDs = env.fromCollection(customTypeData)
+
+    // should not work, field position key on custom type
+    customDs.groupBy(0)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testGroupByKeyFields4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should not work, fiels position out of range
+    tupleDs.groupBy(5)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testGroupByKeyFields5(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should not work, negative field position
+    tupleDs.groupBy(-1)
+  }
+
+  @Ignore
+  @Test
+  def testGroupByKeyExpressions1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds = env.fromCollection(customTypeData)
+
+    // should work
+    try {
+//      ds.groupBy("i");
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Ignore
+  @Test(expected = classOf[UnsupportedOperationException])
+  def testGroupByKeyExpressions2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+
+    // should not work: groups on basic type
+//    longDs.groupBy("l");
+    val longDs = env.fromCollection(emptyLongData)
+  }
+
+  @Ignore
+  @Test(expected = classOf[InvalidProgramException])
+  def testGroupByKeyExpressions3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val customDs = env.fromCollection(customTypeData)
+
+    // should not work: groups on custom type
+    customDs.groupBy(0)
+  }
+
+  @Ignore
+  @Test(expected = classOf[IllegalArgumentException])
+  def testGroupByKeyExpressions4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds = env.fromCollection(customTypeData)
+
+    // should not work, non-existent field
+//    ds.groupBy("myNonExistent");
+  }
+
+  @Test
+  def testGroupByKeySelector1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    try {
+      val customDs = env.fromCollection(customTypeData)
+      customDs.groupBy { _.l }
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
+  def testGroupSortKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+    try {
+      tupleDs.groupBy(0).sortGroup(0, Order.ASCENDING)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testGroupSortKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+
+    // should not work, field position out of range
+    tupleDs.groupBy(0).sortGroup(5, Order.ASCENDING)
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testGroupSortKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val longDs = env.fromCollection(emptyLongData)
+    longDs.groupBy { x: Long => x } .sortGroup(0, Order.ASCENDING)
+  }
+
+  @Test
+  def testChainedGroupSortKeyFields(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tupleDs = env.fromCollection(emptyTupleData)
+    try {
+      tupleDs.groupBy(0).sortGroup(0, Order.ASCENDING).sortGroup(2, Order.DESCENDING)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
new file mode 100644
index 0000000..fff9857
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators
+
+import org.junit.Assert
+import org.apache.flink.api.common.InvalidProgramException
+import org.junit.Ignore
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+class JoinOperatorTest {
+
+  private val emptyTupleData = Array[(Int, Long, String, Long, Int)]()
+  private val customTypeData = Array[CustomType](new CustomType())
+  private val emptyLongData = Array[Long]()
+
+  @Test
+  def testJoinKeyFields1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+    try {
+      ds1.join(ds2).where(0).equalTo(0)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyFields2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+    ds1.join(ds2).where(0).equalTo(2)
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyFields3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+    ds1.join(ds2).where(0, 1).equalTo(2)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testJoinKeyFields4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+    ds1.join(ds2).where(5).equalTo(0)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testJoinKeyFields5(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(emptyTupleData)
+    ds1.join(ds2).where(-1).equalTo(-1)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testJoinKeyFields6(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+    ds1.join(ds2).where(5).equalTo(0)
+  }
+
+  @Ignore
+  @Test
+  def testJoinKeyExpressions1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should work
+    try {
+//      ds1.join(ds2).where("i").equalTo("i")
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Ignore
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyExpressions2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, incompatible join key types
+//    ds1.join(ds2).where("i").equalTo("s")
+  }
+
+  @Ignore
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyExpressions3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, incompatible number of keys
+//    ds1.join(ds2).where("i", "s").equalTo("i")
+  }
+
+  @Ignore
+  @Test(expected = classOf[IllegalArgumentException])
+  def testJoinKeyExpressions4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, join key non-existent
+//    ds1.join(ds2).where("myNonExistent").equalTo("i")
+  }
+
+  @Test
+  def testJoinKeySelectors1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should work
+    try {
+      ds1.join(ds2).where { _.l} equalTo { _.l }
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
+  def testJoinKeyMixing1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(customTypeData)
+    val ds2 = env.fromCollection(emptyTupleData)
+
+    // should work
+    try {
+      ds1.join(ds2).where { _.l }.equalTo(3)
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test
+  def testJoinKeyMixing2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should work
+    try {
+      ds1.join(ds2).where(3).equalTo { _.l }
+    }
+    catch {
+      case e: Exception => Assert.fail()
+    }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyMixing3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, incompatible types
+    ds1.join(ds2).where(2).equalTo { _.l }
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testJoinKeyMixing4(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val ds1 = env.fromCollection(emptyTupleData)
+    val ds2 = env.fromCollection(customTypeData)
+
+    // should not work, more than one field position key
+    ds1.join(ds2).where(1, 3) equalTo { _.l }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
new file mode 100644
index 0000000..7501f82
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators.translation
+
+import org.apache.flink.api.common.Plan
+import org.apache.flink.api.common.operators.base.{GenericDataSourceBase, GroupReduceOperatorBase}
+
+import org.apache.flink.api.java.aggregation.Aggregations
+import org.apache.flink.api.scala._
+import org.junit.Assert.{assertEquals, assertTrue, fail}
+import org.junit.Test
+
+class AggregateTranslationTest {
+  @Test
+  def translateAggregate(): Unit =  {
+    try {
+      val DOP = 8
+
+      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+
+      val initialData = env.fromElements((3.141592, "foobar", 77L))
+
+      initialData.groupBy(0).aggregate(Aggregations.MIN, 1).aggregate(Aggregations.SUM, 2).print()
+
+      val p: Plan = env.createProgramPlan()
+      val sink = p.getDataSinks.iterator.next
+
+      val reducer= sink.getInput.asInstanceOf[GroupReduceOperatorBase[_, _, _]]
+
+      assertEquals(1, reducer.getKeyColumns(0).length)
+      assertEquals(0, reducer.getKeyColumns(0)(0))
+      assertEquals(-1, reducer.getDegreeOfParallelism)
+      assertTrue(reducer.isCombinable)
+      assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Test caused an error: " + e.getMessage)
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
new file mode 100644
index 0000000..5631cbb
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
@@ -0,0 +1,250 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators.translation
+
+import org.apache.flink.api.common.functions.{RichCoGroupFunction, RichMapFunction,
+RichJoinFunction}
+import org.apache.flink.api.java.operators.translation.WrappingFunction
+import org.junit.Assert.assertArrayEquals
+import org.junit.Assert.assertEquals
+import org.junit.Assert.fail
+import org.apache.flink.api.common.{InvalidProgramException, Plan}
+import org.apache.flink.api.common.aggregators.LongSumAggregator
+import org.apache.flink.api.common.operators.base.DeltaIterationBase
+import org.apache.flink.api.common.operators.base.GenericDataSinkBase
+import org.apache.flink.api.common.operators.base.JoinOperatorBase
+import org.apache.flink.api.common.operators.base.MapOperatorBase
+import org.junit.Test
+
+import org.apache.flink.util.Collector
+
+import org.apache.flink.api.scala._
+
+class DeltaIterationTranslationTest {
+
+  @Test
+  def testCorrectTranslation(): Unit = {
+    try {
+      val JOB_NAME = "Test JobName"
+      val ITERATION_NAME = "Test Name"
+      val BEFORE_NEXT_WORKSET_MAP = "Some Mapper"
+      val AGGREGATOR_NAME = "AggregatorName"
+      val ITERATION_KEYS = Array(2)
+      val NUM_ITERATIONS = 13
+      val DEFAULT_DOP = 133
+      val ITERATION_DOP = 77
+
+      val env = ExecutionEnvironment.getExecutionEnvironment
+      env.setDegreeOfParallelism(DEFAULT_DOP)
+
+      val initialSolutionSet = env.fromElements((3.44, 5L, "abc"))
+      val initialWorkSet = env.fromElements((1.23, "abc"))
+
+      val result = initialSolutionSet.iterateDelta(initialWorkSet, NUM_ITERATIONS, ITERATION_KEYS) {
+        (s, ws) =>
+          val wsSelfJoin = ws.map(new IdentityMapper[(Double, String)]())
+            .join(ws).where(1).equalTo(1) { (l, r) => Some(l) }
+
+          val joined = wsSelfJoin.join(s).where(1).equalTo(2).apply(new SolutionWorksetJoin)
+          (joined, joined.map(new NextWorksetMapper).name(BEFORE_NEXT_WORKSET_MAP))
+      }
+      result.name(ITERATION_NAME)
+        .setParallelism(ITERATION_DOP)
+        .registerAggregator(AGGREGATOR_NAME, new LongSumAggregator)
+
+      result.print()
+      result.writeAsText("/dev/null")
+
+      val p: Plan = env.createProgramPlan(JOB_NAME)
+      assertEquals(JOB_NAME, p.getJobName)
+      assertEquals(DEFAULT_DOP, p.getDefaultParallelism)
+      var sink1: GenericDataSinkBase[_] = null
+      var sink2: GenericDataSinkBase[_] = null
+      val sinks = p.getDataSinks.iterator
+      sink1 = sinks.next
+      sink2 = sinks.next
+
+      val iteration: DeltaIterationBase[_, _] =
+        sink1.getInput.asInstanceOf[DeltaIterationBase[_,_]]
+
+      assertEquals(iteration, sink2.getInput)
+      assertEquals(NUM_ITERATIONS, iteration.getMaximumNumberOfIterations)
+      assertArrayEquals(ITERATION_KEYS, iteration.getSolutionSetKeyFields)
+      assertEquals(ITERATION_DOP, iteration.getDegreeOfParallelism)
+      assertEquals(ITERATION_NAME, iteration.getName)
+
+      val nextWorksetMapper: MapOperatorBase[_, _, _] =
+        iteration.getNextWorkset.asInstanceOf[MapOperatorBase[_, _, _]]
+      val solutionSetJoin: JoinOperatorBase[_, _, _, _] =
+        iteration.getSolutionSetDelta.asInstanceOf[JoinOperatorBase[_, _, _, _]]
+      val worksetSelfJoin: JoinOperatorBase[_, _, _, _] =
+        solutionSetJoin.getFirstInput.asInstanceOf[JoinOperatorBase[_, _, _, _]]
+      val worksetMapper: MapOperatorBase[_, _, _] =
+        worksetSelfJoin.getFirstInput.asInstanceOf[MapOperatorBase[_, _, _]]
+
+      assertEquals(classOf[IdentityMapper[_]], worksetMapper.getUserCodeWrapper.getUserCodeClass)
+
+
+      assertEquals(classOf[NextWorksetMapper], nextWorksetMapper.getUserCodeWrapper.getUserCodeClass)
+
+
+      if (solutionSetJoin.getUserCodeWrapper.getUserCodeObject.isInstanceOf[WrappingFunction[_]]) {
+        val wf: WrappingFunction[_] = solutionSetJoin.getUserCodeWrapper.getUserCodeObject
+          .asInstanceOf[WrappingFunction[_]]
+        assertEquals(classOf[SolutionWorksetJoin],
+          wf.getWrappedFunction.getClass)
+      }
+      else {
+        assertEquals(classOf[SolutionWorksetJoin],
+          solutionSetJoin.getUserCodeWrapper.getUserCodeClass)
+      }
+
+      assertEquals(BEFORE_NEXT_WORKSET_MAP, nextWorksetMapper.getName)
+      assertEquals(AGGREGATOR_NAME, iteration.getAggregators.getAllRegisteredAggregators.iterator
+        .next.getName)
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail(e.getMessage)
+      }
+    }
+  }
+
+  @Test
+  def testRejectWhenSolutionSetKeysDontMatchJoin(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+
+      val initialSolutionSet = env.fromElements((3.44, 5L, "abc"))
+      val initialWorkSet = env.fromElements((1.23, "abc"))
+
+      val iteration = initialSolutionSet.iterateDelta(initialWorkSet, 10, Array(0)) {
+        (s, ws) =>
+          try {
+            ws.join(s).where(1).equalTo(2)
+            fail("Accepted invalid program.")
+          } catch {
+            case e: InvalidProgramException => // all good
+          }
+          try {
+            s.join(ws).where(2).equalTo(1)
+            fail("Accepted invalid program.")
+          } catch {
+            case e: InvalidProgramException => // all good
+          }
+          (s, ws)
+      }
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail(e.getMessage)
+      }
+    }
+  }
+
+  @Test
+  def testRejectWhenSolutionSetKeysDontMatchCoGroup(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+
+      val initialSolutionSet = env.fromElements((3.44, 5L, "abc"))
+      val initialWorkSet = env.fromElements((1.23, "abc"))
+
+      val iteration = initialSolutionSet.iterateDelta(initialWorkSet, 10, Array(0)) {
+        (s, ws) =>
+          try {
+            ws.coGroup(s).where(1).equalTo(2)
+            fail("Accepted invalid program.")
+          } catch {
+            case e: InvalidProgramException => // all good
+          }
+          try {
+            s.coGroup(ws).where(2).equalTo(1)
+            fail("Accepted invalid program.")
+          } catch {
+            case e: InvalidProgramException => // all good
+          }
+          (s, ws)
+      }
+    } catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail(e.getMessage)
+      }
+    }
+  }
+//
+//  @Test def testRejectWhenSolutionSetKeysDontMatchCoGroup {
+//    try {
+//      val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+//      @SuppressWarnings(Array("unchecked")) val initialSolutionSet: DataSet[Tuple3[Double, Long,
+//        String]] = env.fromElements(new Tuple3[Double, Long, String](3.44, 5L, "abc"))
+//      @SuppressWarnings(Array("unchecked")) val initialWorkSet: DataSet[Tuple2[Double,
+//        String]] = env.fromElements(new Tuple2[Double, String](1.23, "abc"))
+//      val iteration: DeltaIteration[Tuple3[Double, Long, String], Tuple2[Double,
+//        String]] = initialSolutionSet.iterateDelta(initialWorkSet, 10, 1)
+//      try {
+//        iteration.getWorkset.coGroup(iteration.getSolutionSet).where(1).equalTo(2).`with`(new DeltaIterationTranslationTest.SolutionWorksetCoGroup1)
+//        fail("Accepted invalid program.")
+//      }
+//      catch {
+//        case e: InvalidProgramException => {
+//        }
+//      }
+//      try {
+//        iteration.getSolutionSet.coGroup(iteration.getWorkset).where(2).equalTo(1).`with`(new DeltaIterationTranslationTest.SolutionWorksetCoGroup2)
+//        fail("Accepted invalid program.")
+//      }
+//      catch {
+//        case e: InvalidProgramException => {
+//        }
+//      }
+//    }
+//    catch {
+//      case e: Exception => {
+//        System.err.println(e.getMessage)
+//        e.printStackTrace
+//        fail(e.getMessage)
+//      }
+//    }
+//  }
+}
+
+class SolutionWorksetJoin
+  extends RichJoinFunction[(Double, String), (Double, Long, String), (Double, Long, String)] {
+  def join(first: (Double, String), second: (Double, Long, String)): (Double, Long, String) = {
+    null
+  }
+}
+
+class NextWorksetMapper extends RichMapFunction[(Double, Long, String), (Double, String)] {
+  def map(value: (Double, Long, String)): (Double, String) = {
+    null
+  }
+}
+
+class IdentityMapper[T] extends RichMapFunction[T, T] {
+  def map(value: T): T = {
+    value
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DistinctTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DistinctTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DistinctTranslationTest.scala
new file mode 100644
index 0000000..4ba9652
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/DistinctTranslationTest.scala
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators.translation
+
+import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase
+import org.junit.Assert
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+class DistinctTranslationTest {
+  @Test
+  def testCombinable(): Unit = {
+    try {
+      val env = ExecutionEnvironment.getExecutionEnvironment
+      val input = env.fromElements("1", "2", "1", "3")
+
+      val op = input.distinct { x => x}
+      op.print()
+
+      val p = env.createProgramPlan()
+
+      val reduceOp =
+        p.getDataSinks.iterator.next.getInput.asInstanceOf[GroupReduceOperatorBase[_, _, _]]
+
+      Assert.assertTrue(reduceOp.isCombinable)
+    }
+    catch {
+      case e: Exception => {
+        e.printStackTrace()
+        Assert.fail(e.getMessage)
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
new file mode 100644
index 0000000..36aa4f2
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.operators.translation
+
+import org.apache.flink.api.java.operators.translation.{KeyExtractingMapper,
+PlanUnwrappingReduceOperator}
+import org.apache.flink.api.java.typeutils.{TupleTypeInfo, BasicTypeInfo}
+import org.junit.Assert._
+import org.apache.flink.api.common.operators.base.GenericDataSinkBase
+import org.apache.flink.api.common.operators.base.GenericDataSourceBase
+import org.apache.flink.api.common.operators.base.MapOperatorBase
+import org.apache.flink.api.common.operators.base.ReduceOperatorBase
+import org.junit.Test
+
+import org.apache.flink.api.scala._
+
+class ReduceTranslationTest {
+  @Test
+  def translateNonGroupedReduce(): Unit = {
+    try {
+      val DOP = 8
+      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+
+      val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1)
+
+
+      initialData reduce { (v1, v2) => v1 } print()
+
+      val p = env.createProgramPlan(
+
+)
+      val sink: GenericDataSinkBase[_] = p.getDataSinks.iterator.next
+      val reducer: ReduceOperatorBase[_, _] = sink.getInput.asInstanceOf[ReduceOperatorBase[_, _]]
+
+      assertEquals(initialData.set.getType, reducer.getOperatorInfo.getInputType)
+      assertEquals(initialData.set.getType, reducer.getOperatorInfo.getOutputType)
+      assertTrue(reducer.getKeyColumns(0) == null || reducer.getKeyColumns(0).length == 0)
+      assertTrue(reducer.getDegreeOfParallelism == 1 || reducer.getDegreeOfParallelism == -1)
+      assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Test caused an error: " + e.getMessage)
+      }
+    }
+  }
+
+  @Test
+  def translateGroupedReduceNoMapper(): Unit = {
+    try {
+      val DOP: Int = 8
+      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+
+      val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1)
+
+      initialData.groupBy(2) reduce { (v1, v2) => v1 } print()
+
+      val p = env.createProgramPlan()
+
+      val sink: GenericDataSinkBase[_] = p.getDataSinks.iterator.next
+      val reducer: ReduceOperatorBase[_, _] = sink.getInput.asInstanceOf[ReduceOperatorBase[_, _]]
+      assertEquals(initialData.set.getType, reducer.getOperatorInfo.getInputType)
+      assertEquals(initialData.set.getType, reducer.getOperatorInfo.getOutputType)
+      assertTrue(reducer.getDegreeOfParallelism == DOP || reducer.getDegreeOfParallelism == -1)
+      assertArrayEquals(Array[Int](2), reducer.getKeyColumns(0))
+      assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Test caused an error: " + e.getMessage)
+      }
+    }
+  }
+
+  @Test
+  def translateGroupedReduceWithKeyExtractor(): Unit = {
+    try {
+      val DOP: Int = 8
+      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+
+      val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1)
+
+      initialData.groupBy { _._2 }. reduce { (v1, v2) => v1 } setParallelism(4) print()
+
+      val p = env.createProgramPlan()
+      val sink: GenericDataSinkBase[_] = p.getDataSinks.iterator.next
+      val keyProjector: MapOperatorBase[_, _, _] = sink.getInput.asInstanceOf[MapOperatorBase[_,
+        _, _]]
+      val reducer: PlanUnwrappingReduceOperator[_, _] = keyProjector.getInput
+        .asInstanceOf[PlanUnwrappingReduceOperator[_, _]]
+      val keyExtractor: MapOperatorBase[_, _, _] = reducer.getInput
+        .asInstanceOf[MapOperatorBase[_, _, _]]
+      assertEquals(1, keyExtractor.getDegreeOfParallelism)
+      assertEquals(4, reducer.getDegreeOfParallelism)
+      assertEquals(4, keyProjector.getDegreeOfParallelism)
+      val keyValueInfo = new TupleTypeInfo(
+        BasicTypeInfo.STRING_TYPE_INFO,
+        createTypeInformation[(Double, String, Long)])
+      assertEquals(initialData.set.getType, keyExtractor.getOperatorInfo.getInputType)
+      assertEquals(keyValueInfo, keyExtractor.getOperatorInfo.getOutputType)
+      assertEquals(keyValueInfo, reducer.getOperatorInfo.getInputType)
+      assertEquals(keyValueInfo, reducer.getOperatorInfo.getOutputType)
+      assertEquals(keyValueInfo, keyProjector.getOperatorInfo.getInputType)
+      assertEquals(initialData.set.getType, keyProjector.getOperatorInfo.getOutputType)
+      assertEquals(classOf[KeyExtractingMapper[_, _]], keyExtractor.getUserCodeWrapper.getUserCodeClass)
+      assertTrue(keyExtractor.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
+    }
+    catch {
+      case e: Exception => {
+        System.err.println(e.getMessage)
+        e.printStackTrace()
+        fail("Test caused an error: " + e.getMessage)
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
new file mode 100644
index 0000000..95a1b77
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/GenericPairComparatorTest.scala
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.api.common.typeutils.base.{DoubleComparator, DoubleSerializer, IntComparator, IntSerializer}
+
+import org.apache.flink.api.java.typeutils.runtime.{GenericPairComparator, TupleComparator}
+import org.apache.flink.api.scala.runtime.tuple.base.PairComparatorTestBase
+import org.apache.flink.api.scala.typeutils.ScalaTupleComparator
+
+class GenericPairComparatorTest
+  extends PairComparatorTestBase[(Int, String, Double), (Int, Float, Long, Double)] {
+
+  override protected def createComparator(ascending: Boolean):
+  GenericPairComparator[(Int, String, Double), (Int, Float, Long, Double)] = {
+    val fields1  = Array[Int](0, 2)
+    val fields2 = Array[Int](0, 3)
+
+    val comps1 =
+      Array[TypeComparator[_]](new IntComparator(ascending), new DoubleComparator(ascending))
+    val comps2 =
+      Array[TypeComparator[_]](new IntComparator(ascending), new DoubleComparator(ascending))
+
+    val sers1 =
+      Array[TypeSerializer[_]](IntSerializer.INSTANCE, DoubleSerializer.INSTANCE)
+    val sers2 =
+      Array[TypeSerializer[_]](IntSerializer.INSTANCE, DoubleSerializer.INSTANCE)
+
+    val comp1 = new ScalaTupleComparator[(Int, String, Double)](fields1, comps1, sers1)
+    val comp2 = new ScalaTupleComparator[(Int, Float, Long, Double)](fields2, comps2, sers2)
+
+    new GenericPairComparator[(Int, String, Double), (Int, Float, Long, Double)](comp1, comp2)
+  }
+
+  protected def getSortedTestData:
+  (Array[(Int, String, Double)], Array[(Int, Float, Long, Double)]) = {
+    (dataISD, dataIDL)
+  }
+
+  private val dataISD = Array(
+    (4, "hello", 20.0),
+    (4, "world", 23.2),
+    (5, "hello", 18.0),
+    (5, "world", 19.2),
+    (6, "hello", 16.0),
+    (6, "world", 17.2),
+    (7, "hello", 14.0),
+    (7,"world", 15.2))
+
+  private val dataIDL = Array(
+    (4, 0.11f, 14L, 20.0),
+    (4, 0.221f, 15L, 23.2),
+    (5, 0.33f, 15L, 18.0),
+    (5, 0.44f, 20L, 19.2),
+    (6, 0.55f, 20L, 16.0),
+    (6, 0.66f, 29L, 17.2),
+    (7, 0.77f, 29L, 14.0),
+    (7, 0.88f, 34L, 15.2))
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala
new file mode 100644
index 0000000..8e23744
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+import org.apache.flink.api.scala._
+
+
+class TupleComparatorILD2Test extends TupleComparatorTestBase[(Int, Long, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]]
+      .createComparator(Array(0, 1), Array(ascending, ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, Long, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, 14L, 20.0),
+    (4, 15L, 23.2),
+    (5, 15L, 20.0),
+    (5, 20L, 20.0),
+    (6, 20L, 23.2),
+    (6, 29L, 20.0),
+    (7, 29L, 20.0),
+    (7, 34L, 23.2)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala
new file mode 100644
index 0000000..6907165
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+
+class TupleComparatorILD3Test extends TupleComparatorTestBase[(Int, Long, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]]
+      .createComparator(Array(0, 1, 2), Array(ascending, ascending, ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, Long, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, 4L, 20.0),
+    (4, 4L, 23.2),
+    (4, 9L, 20.0),
+    (5, 4L, 20.0),
+    (5, 4L, 23.2),
+    (5, 9L, 20.0),
+    (6, 4L, 20.0),
+    (6, 4L, 23.2)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala
new file mode 100644
index 0000000..445aaa1
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+
+class TupleComparatorILDC3Test extends TupleComparatorTestBase[(Int, Long, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]]
+      .createComparator(Array(2, 0, 1), Array(ascending, ascending, ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, Long, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, 4L, 20.0),
+    (5, 1L, 20.0),
+    (5, 2L, 20.0),
+    (5, 10L, 23.0),
+    (5, 19L, 24.0),
+    (5, 20L, 24.0),
+    (5, 24L, 25.0),
+    (5, 25L, 25.0)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala
new file mode 100644
index 0000000..518705f
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+import org.apache.flink.api.scala._
+
+class TupleComparatorILDX1Test extends TupleComparatorTestBase[(Int, Long, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]]
+      .createComparator(Array(1), Array(ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, Long, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, 4L, 20.0),
+    (4, 5L, 23.2),
+    (4, 9L, 20.0),
+    (4, 10L, 24.0),
+    (4, 19L, 23.2),
+    (4, 20L, 24.0),
+    (4, 24L, 20.0),
+    (4, 25L, 23.2)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala
new file mode 100644
index 0000000..0976da8
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+import org.apache.flink.api.scala._
+
+class TupleComparatorILDXC2Test extends TupleComparatorTestBase[(Int, Long, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]]
+      .createComparator(Array(2, 1), Array(ascending, ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, Long, Double)] = {
+    val ti = createTypeInformation[(Int, Long, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, Long, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, 4L, 20.0),
+    (4, 5L, 20.0),
+    (4, 3L, 23.0),
+    (4, 19L, 23.0),
+    (4, 17L, 24.0),
+    (4, 18L, 24.0),
+    (4, 24L, 25.0),
+    (4, 25L, 25.0)
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b8131fa7/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala
new file mode 100644
index 0000000..7478701
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.runtime
+
+import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator}
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase
+
+class TupleComparatorISD1Test extends TupleComparatorTestBase[(Int, String, Double)] {
+
+  protected def createComparator(ascending: Boolean): TypeComparator[(Int, String, Double)] = {
+    val ti = createTypeInformation[(Int, String, Double)]
+    ti.asInstanceOf[TupleTypeInfoBase[(Int, String, Double)]]
+      .createComparator(Array(0), Array(ascending))
+  }
+
+  protected def createSerializer: TypeSerializer[(Int, String, Double)] = {
+    val ti = createTypeInformation[(Int, String, Double)]
+    ti.createSerializer()
+  }
+
+  protected def getSortedTestData: Array[(Int, String, Double)] = {
+    dataISD
+  }
+
+  private val dataISD = Array(
+    (4, "hello", 20.0),
+    (5, "hello", 23.2),
+    (6, "world", 20.0),
+    (7, "hello", 20.0),
+    (8, "hello", 23.2),
+    (9, "world", 20.0),
+    (10, "hello", 20.0),
+    (11, "hello", 23.2)
+  )
+}
+


[40/60] git commit: [scala] Reactivate DeltaIterationSanityCheckTest

Posted by al...@apache.org.
[scala] Reactivate DeltaIterationSanityCheckTest


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

Branch: refs/heads/master
Commit: fd2809813a0b98e283d0c3e566fcc56425566ca2
Parents: 31ed0c4
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 16 12:16:18 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 .../scala/DeltaIterationSanityCheckTest.scala   | 361 +++++++++----------
 1 file changed, 169 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fd280981/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
index d5b0d24..094c1b4 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/DeltaIterationSanityCheckTest.scala
@@ -1,192 +1,169 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.api.scala
-//
-//import org.junit.Test
-//import org.apache.flink.api.common.InvalidProgramException
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//import org.scalatest.junit.AssertionsForJUnit
-//
-//// Verify that the sanity checking in delta iterations works. We just
-//// have a dummy job that is not meant to be executed. Only verify that
-//// the join/coGroup inside the iteration is checked.
-//class DeltaIterationSanityCheckTest extends Serializable {
-//
-//  @Test
-//  def testCorrectJoinWithSolution1 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = s join ws where {_._1} isEqualTo {_._1} map { (l, r) => l }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test
-//  def testCorrectJoinWithSolution2 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = ws join s where {_._1} isEqualTo {_._1} map { (l, r) => l }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test(expected = classOf[InvalidProgramException])
-//  def testIncorrectJoinWithSolution1 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = s join ws where {_._2} isEqualTo {_._2} map { (l, r) => l }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test(expected = classOf[InvalidProgramException])
-//  def testIncorrectJoinWithSolution2 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = ws join s where {_._2} isEqualTo {_._2} map { (l, r) => l }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test(expected = classOf[InvalidProgramException])
-//  def testIncorrectJoinWithSolution3 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = s join ws where {_._1} isEqualTo {_._1} map { (l, r) => l }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._2}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//   }
-//
-//  @Test
-//  def testCorrectCoGroupWithSolution1 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = s cogroup ws where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test
-//  def testCorrectCoGroupWithSolution2 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = ws cogroup s where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test(expected = classOf[InvalidProgramException])
-//  def testIncorrectCoGroupWithSolution1 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = s cogroup ws where {_._2} isEqualTo {_._2} map { (l, r) => l.next() }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test(expected = classOf[InvalidProgramException])
-//  def testIncorrectCoGroupWithSolution2 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = ws cogroup s where {_._2} isEqualTo {_._2} map { (l, r) => l.next() }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._1}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//
-//  @Test(expected = classOf[InvalidProgramException])
-//  def testIncorrectCoGroupWithSolution3 {
-//    val solutionInput = CollectionDataSource(Array((1, "1")))
-//    val worksetInput = CollectionDataSource(Array((2, "2")))
-//
-//    def step(s: DataSetOLD[(Int, String)], ws: DataSetOLD[(Int, String)]) = {
-//      val result = s cogroup ws where {_._1} isEqualTo {_._1} map { (l, r) => l.next() }
-//      (result, ws)
-//    }
-//    val iteration = solutionInput.iterateWithDelta(worksetInput, {_._2}, step, 10)
-//
-//    val output = iteration.write("/dummy", CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output))
-//  }
-//}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala
+
+import org.junit.Test
+import org.apache.flink.api.common.InvalidProgramException
+
+import org.apache.flink.api.scala._
+import org.scalatest.junit.AssertionsForJUnit
+
+// Verify that the sanity checking in delta iterations works. We just
+// have a dummy job that is not meant to be executed. Only verify that
+// the join/coGroup inside the iteration is checked.
+class DeltaIterationSanityCheckTest extends Serializable {
+
+  @Test
+  def testCorrectJoinWithSolution1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = s.join(ws).where("_1").equalTo("_1") { (l, r) => l }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+
+  @Test
+  def testCorrectJoinWithSolution2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = ws.join(s).where("_1").equalTo("_1") { (l, r) => l }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testIncorrectJoinWithSolution1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = s.join(ws).where("_2").equalTo("_2") { (l, r) => l }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testIncorrectJoinWithSolution2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = ws.join(s).where("_2").equalTo("_2") { (l, r) => l }
+      (result, ws)
+    }
+
+    val output = iteration.print()  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testIncorrectJoinWithSolution3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_2")) { (s, ws) =>
+      val result = ws.join(s).where("_1").equalTo("_1") { (l, r) => l }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+   }
+
+  @Test
+  def testCorrectCoGroupWithSolution1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = s.coGroup(ws).where("_1").equalTo("_1") { (l, r) => l.min }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+
+  @Test
+  def testCorrectCoGroupWithSolution2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = ws.coGroup(s).where("_1").equalTo("_1") { (l, r) => l.min }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testIncorrectCoGroupWithSolution1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = s.coGroup(ws).where("_2").equalTo("_2") { (l, r) => l.min }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testIncorrectCoGroupWithSolution2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_1")) { (s, ws) =>
+      val result = ws.coGroup(s).where("_2").equalTo("_2") { (l, r) => l.min }
+      (result, ws)
+    }
+
+    val output = iteration.print()  }
+
+  @Test(expected = classOf[InvalidProgramException])
+  def testIncorrectCoGroupWithSolution3(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val solutionInput = env.fromElements((1, "1"))
+    val worksetInput = env.fromElements((2, "2"))
+
+    val iteration = solutionInput.iterateDelta(worksetInput, 10, Array("_2")) { (s, ws) =>
+      val result = ws.coGroup(s).where("_1").equalTo("_1") { (l, r) => l.min }
+      (result, ws)
+    }
+
+    val output = iteration.print()
+  }
+}


[26/60] Renamed java examples package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
deleted file mode 100644
index 10cf748..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/util/PageRankData.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.graph.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-/**
- * Provides the default data sets used for the PageRank example program.
- * The default data sets are used, if no parameters are given to the program.
- *
- */
-public class PageRankData {
-
-	public static final Object[][] EDGES = {
-		{1L, 2L},
-		{1L, 15L},
-		{2L, 3L},
-		{2L, 4L},
-		{2L, 5L},
-		{2L, 6L},
-		{2L, 7L},
-		{3L, 13L},
-		{4L, 2L},
-		{5L, 11L},
-		{5L, 12L},
-		{6L, 1L},
-		{6L, 7L},
-		{6L, 8L},
-		{7L, 1L},
-		{7L, 8L},
-		{8L, 1L},
-		{8L, 9L},
-		{8L, 10L},
-		{9L, 14L},
-		{9L, 1L},
-		{10L, 1L},
-		{10L, 13L},
-		{11L, 12L},
-		{11L, 1L},
-		{12L, 1L},
-		{13L, 14L},
-		{14L, 12L},
-		{15L, 1L},
-	};
-	
-	private static long numPages = 15;
-	
-	public static DataSet<Tuple2<Long, Long>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-		
-		List<Tuple2<Long, Long>> edges = new ArrayList<Tuple2<Long, Long>>();
-		for(Object[] e : EDGES) {
-			edges.add(new Tuple2<Long, Long>((Long)e[0], (Long)e[1]));
-		}
-		return env.fromCollection(edges);
-	}
-	
-	public static DataSet<Long> getDefaultPagesDataSet(ExecutionEnvironment env) {
-		return env.generateSequence(1, 15);
-	}
-	
-	public static long getNumberOfPages() {
-		return numPages;
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/misc/PiEstimation.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/misc/PiEstimation.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/misc/PiEstimation.java
deleted file mode 100644
index ef336da..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/misc/PiEstimation.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.misc;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/** 
- * Estimates the value of Pi using the Monte Carlo method.
- * The area of a circle is Pi * R^2, R being the radius of the circle 
- * The area of a square is 4 * R^2, where the length of the square's edge is 2*R.
- * 
- * Thus Pi = 4 * (area of circle / area of square).
- * 
- * The idea is to find a way to estimate the circle to square area ratio.
- * The Monte Carlo method suggests collecting random points (within the square)
- * and then counting the number of points that fall within the circle
- * 
- * <pre>
- * {@code
- * x = Math.random()
- * y = Math.random()
- * 
- * x * x + y * y < 1
- * }
- * </pre>
- */
-@SuppressWarnings("serial")
-public class PiEstimation implements java.io.Serializable {
-	
-	
-	public static void main(String[] args) throws Exception {
-
-		final long numSamples = args.length > 0 ? Long.parseLong(args[0]) : 1000000;
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		// count how many of the samples would randomly fall into
-		// the unit circle
-		DataSet<Long> count = 
-				env.generateSequence(1, numSamples)
-				.map(new Sampler())
-				.reduce(new SumReducer());
-
-		// the ratio of the unit circle surface to 4 times the unit square is pi
-		DataSet<Double> pi = count
-				.map(new MapFunction<Long, Double>() {
-					public Double map(Long value) {
-						return value * 4.0 / numSamples;
-					}
-				});
-
-		System.out.println("We estimate Pi to be:");
-		pi.print();
-
-		env.execute();
-	}
-
-	//*************************************************************************
-	//     USER FUNCTIONS
-	//*************************************************************************
-	
-	
-	/** 
-	 * Sampler randomly emits points that fall within a square of edge x * y.
-	 * It calculates the distance to the center of a virtually centered circle of radius x = y = 1
-	 * If the distance is less than 1, then and only then does it returns a 1.
-	 */
-	public static class Sampler implements MapFunction<Long, Long> {
-
-		@Override
-		public Long map(Long value) throws Exception{
-			double x = Math.random();
-			double y = Math.random();
-			return (x * x + y * y) < 1 ? 1L : 0L;
-		}
-	}
-
-	
-	/** 
-	 * Simply sums up all long values.
-	 */
-	public static final class SumReducer implements ReduceFunction<Long>{
-
-		@Override
-		public Long reduce(Long value1, Long value2) throws Exception {
-			return value1 + value2;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
deleted file mode 100644
index 6ef6270..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/LinearRegression.java
+++ /dev/null
@@ -1,316 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.ml;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.example.java.ml.util.LinearRegressionData;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-
-/**
- * This example implements a basic Linear Regression  to solve the y = theta0 + theta1*x problem using batch gradient descent algorithm.
- *
- * <p>
- * Linear Regression with BGD(batch gradient descent) algorithm is an iterative clustering algorithm and works as follows:<br>
- * Giving a data set and target set, the BGD try to find out the best parameters for the data set to fit the target set.
- * In each iteration, the algorithm computes the gradient of the cost function and use it to update all the parameters.
- * The algorithm terminates after a fixed number of iterations (as in this implementation)
- * With enough iteration, the algorithm can minimize the cost function and find the best parameters
- * This is the Wikipedia entry for the <a href = "http://en.wikipedia.org/wiki/Linear_regression">Linear regression</a> and <a href = "http://en.wikipedia.org/wiki/Gradient_descent">Gradient descent algorithm</a>.
- * 
- * <p>
- * This implementation works on one-dimensional data. And find the two-dimensional theta.<br>
- * It find the best Theta parameter to fit the target.
- * 
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Data points are represented as two double values separated by a blank character. The first one represent the X(the training data) and the second represent the Y(target).
- * Data points are separated by newline characters.<br>
- * For example <code>"-0.02 -0.04\n5.3 10.6\n"</code> gives two data points (x=-0.02, y=-0.04) and (x=5.3, y=10.6).
- * </ul>
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li> Bulk iterations
- * <li> Broadcast variables in bulk iterations
- * <li> Custom Java objects (PoJos)
- * </ul>
- */
-@SuppressWarnings("serial")
-public class LinearRegression {
-
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception{
-
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		// set up execution environment
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// get input x data from elements
-		DataSet<Data> data = getDataSet(env);
-
-		// get the parameters from elements
-		DataSet<Params> parameters = getParamsDataSet(env);
-
-		// set number of bulk iterations for SGD linear Regression
-		IterativeDataSet<Params> loop = parameters.iterate(numIterations);
-
-		DataSet<Params> new_parameters = data
-				// compute a single step using every sample
-				.map(new SubUpdate()).withBroadcastSet(loop, "parameters")
-				// sum up all the steps
-				.reduce(new UpdateAccumulator())
-				// average the steps and update all parameters
-				.map(new Update());
-
-		// feed new parameters back into next iteration
-		DataSet<Params> result = loop.closeWith(new_parameters);
-
-		// emit result
-		if(fileOutput) {
-			result.writeAsText(outputPath);
-		} else {
-			result.print();
-		}
-
-		// execute program
-		env.execute("Linear Regression example");
-
-	}
-
-	// *************************************************************************
-	//     DATA TYPES
-	// *************************************************************************
-
-	/**
-	 * A simple data sample, x means the input, and y means the target.
-	 */
-	public static class Data implements Serializable{
-		public double x,y;
-
-		public Data() {};
-
-		public Data(double x ,double y){
-			this.x = x;
-			this.y = y;
-		}
-
-		@Override
-		public String toString() {
-			return "(" + x + "|" + y + ")";
-		}
-
-	}
-
-	/**
-	 * A set of parameters -- theta0, theta1.
-	 */
-	public static class Params implements Serializable{
-
-		private double theta0,theta1;
-
-		public Params(){};
-
-		public Params(double x0, double x1){
-			this.theta0 = x0;
-			this.theta1 = x1;
-		}
-
-		@Override
-		public String toString() {
-			return theta0 + " " + theta1;
-		}
-
-		public double getTheta0() {
-			return theta0;
-		}
-
-		public double getTheta1() {
-			return theta1;
-		}
-
-		public void setTheta0(double theta0) {
-			this.theta0 = theta0;
-		}
-
-		public void setTheta1(double theta1) {
-			this.theta1 = theta1;
-		}
-
-		public Params div(Integer a){
-			this.theta0 = theta0 / a ;
-			this.theta1 = theta1 / a ;
-			return this;
-		}
-
-	}
-
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-
-	/** Converts a Tuple2<Double,Double> into a Data. */
-	public static final class TupleDataConverter implements MapFunction<Tuple2<Double, Double>, Data> {
-
-		@Override
-		public Data map(Tuple2<Double, Double> t) throws Exception {
-			return new Data(t.f0, t.f1);
-		}
-	}
-
-	/** Converts a Tuple2<Double,Double> into a Params. */
-	public static final class TupleParamsConverter implements MapFunction<Tuple2<Double, Double>,Params> {
-
-		@Override
-		public Params map(Tuple2<Double, Double> t)throws Exception {
-			return new Params(t.f0,t.f1);
-		}
-	}
-
-	/**
-	 * Compute a single BGD type update for every parameters.
-	 */
-	public static class SubUpdate extends RichMapFunction<Data,Tuple2<Params,Integer>> {
-
-		private Collection<Params> parameters; 
-
-		private Params parameter;
-
-		private int count = 1;
-
-		/** Reads the parameters from a broadcast variable into a collection. */
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			this.parameters = getRuntimeContext().getBroadcastVariable("parameters");
-		}
-
-		@Override
-		public Tuple2<Params, Integer> map(Data in) throws Exception {
-
-			for(Params p : parameters){
-				this.parameter = p; 
-			}
-
-			double theta_0 = parameter.theta0 - 0.01*((parameter.theta0 + (parameter.theta1*in.x)) - in.y);
-			double theta_1 = parameter.theta1 - 0.01*(((parameter.theta0 + (parameter.theta1*in.x)) - in.y) * in.x);
-
-			return new Tuple2<Params,Integer>(new Params(theta_0,theta_1),count);
-		}
-	}
-
-	/**  
-	 * Accumulator all the update.
-	 * */
-	public static class UpdateAccumulator implements ReduceFunction<Tuple2<Params, Integer>> {
-
-		@Override
-		public Tuple2<Params, Integer> reduce(Tuple2<Params, Integer> val1, Tuple2<Params, Integer> val2) {
-
-			double new_theta0 = val1.f0.theta0 + val2.f0.theta0;
-			double new_theta1 = val1.f0.theta1 + val2.f0.theta1;
-			Params result = new Params(new_theta0,new_theta1);
-			return new Tuple2<Params, Integer>( result, val1.f1 + val2.f1);
-
-		}
-	}
-
-	/**
-	 * Compute the final update by average them.
-	 */
-	public static class Update implements MapFunction<Tuple2<Params, Integer>,Params> {
-
-		@Override
-		public Params map(Tuple2<Params, Integer> arg0) throws Exception {
-
-			return arg0.f0.div(arg0.f1);
-
-		}
-
-	}
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String dataPath = null;
-	private static String outputPath = null;
-	private static int numIterations = 10;
-
-	private static boolean parseParameters(String[] programArguments) {
-
-		if(programArguments.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if(programArguments.length == 3) {
-				dataPath = programArguments[0];
-				outputPath = programArguments[1];
-				numIterations = Integer.parseInt(programArguments[2]);
-			} else {
-				System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing Linear Regression example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  We provide a data generator to create synthetic input files for this program.");
-			System.out.println("  Usage: LinearRegression <data path> <result path> <num iterations>");
-		}
-		return true;
-	}
-
-	private static DataSet<Data> getDataSet(ExecutionEnvironment env) {
-		if(fileOutput) {
-			// read data from CSV file
-			return env.readCsvFile(dataPath)
-					.fieldDelimiter(' ')
-					.includeFields(true, true)
-					.types(Double.class, Double.class)
-					.map(new TupleDataConverter());
-		} else {
-			return LinearRegressionData.getDefaultDataDataSet(env);
-		}
-	}
-
-	private static DataSet<Params> getParamsDataSet(ExecutionEnvironment env) {
-
-		return LinearRegressionData.getDefaultParamsDataSet(env);
-
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
deleted file mode 100644
index 31e71f5..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionData.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.ml.util;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.ml.LinearRegression.Data;
-import org.apache.flink.example.java.ml.LinearRegression.Params;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the Linear Regression example
- * program. The default data sets are used, if no parameters are given to the
- * program.
- */
-public class LinearRegressionData {
-
-	// We have the data as object arrays so that we can also generate Scala Data
-	// Sources from it.
-	public static final Object[][] PARAMS = new Object[][] { new Object[] {
-			0.0, 0.0 } };
-
-	public static final Object[][] DATA = new Object[][] {
-			new Object[] { 0.5, 1.0 }, new Object[] { 1.0, 2.0 },
-			new Object[] { 2.0, 4.0 }, new Object[] { 3.0, 6.0 },
-			new Object[] { 4.0, 8.0 }, new Object[] { 5.0, 10.0 },
-			new Object[] { 6.0, 12.0 }, new Object[] { 7.0, 14.0 },
-			new Object[] { 8.0, 16.0 }, new Object[] { 9.0, 18.0 },
-			new Object[] { 10.0, 20.0 }, new Object[] { -0.08, -0.16 },
-			new Object[] { 0.13, 0.26 }, new Object[] { -1.17, -2.35 },
-			new Object[] { 1.72, 3.45 }, new Object[] { 1.70, 3.41 },
-			new Object[] { 1.20, 2.41 }, new Object[] { -0.59, -1.18 },
-			new Object[] { 0.28, 0.57 }, new Object[] { 1.65, 3.30 },
-			new Object[] { -0.55, -1.08 } };
-
-	public static DataSet<Params> getDefaultParamsDataSet(
-			ExecutionEnvironment env) {
-		List<Params> paramsList = new LinkedList<Params>();
-		for (Object[] params : PARAMS) {
-			paramsList.add(new Params((Double) params[0], (Double) params[1]));
-		}
-		return env.fromCollection(paramsList);
-	}
-
-	public static DataSet<Data> getDefaultDataDataSet(ExecutionEnvironment env) {
-
-		List<Data> dataList = new LinkedList<Data>();
-		for (Object[] data : DATA) {
-			dataList.add(new Data((Double) data[0], (Double) data[1]));
-		}
-		return env.fromCollection(dataList);
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionDataGenerator.java
deleted file mode 100644
index 28001ba..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/ml/util/LinearRegressionDataGenerator.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.ml.util;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.text.DecimalFormat;
-import java.util.Locale;
-import java.util.Random;
-
-/**
- * Generates data for the {@link org.apache.flink.example.java.ml.LinearRegression} example program.
- */
-public class LinearRegressionDataGenerator {
-
-	static {
-		Locale.setDefault(Locale.US);
-	}
-
-	private static final String POINTS_FILE = "data";
-	private static final long DEFAULT_SEED = 4650285087650871364L;
-	private static final int DIMENSIONALITY = 1;
-	private static final DecimalFormat FORMAT = new DecimalFormat("#0.00");
-	private static final char DELIMITER = ' ';
-
-	/**
-	 * Main method to generate data for the {@link org.apache.flink.example.java.ml.LinearRegression} example program.
-	 * <p>
-	 * The generator creates to files:
-	 * <ul>
-	 * <li><code>{tmp.dir}/data</code> for the data points
-	 * </ul> 
-	 * 
-	 * @param args 
-	 * <ol>
-	 * <li>Int: Number of data points
-	 * <li><b>Optional</b> Long: Random seed
-	 * </ol>
-	 */
-	public static void main(String[] args) throws IOException {
-
-		// check parameter count
-		if (args.length < 1) {
-			System.out.println("LinearRegressionDataGenerator <numberOfDataPoints> [<seed>]");
-			System.exit(1);
-		}
-
-		// parse parameters
-		final int numDataPoints = Integer.parseInt(args[0]);
-		final long firstSeed = args.length > 1 ? Long.parseLong(args[4]) : DEFAULT_SEED;
-		final Random random = new Random(firstSeed);
-		final String tmpDir = System.getProperty("java.io.tmpdir");
-
-		// write the points out
-		BufferedWriter pointsOut = null;
-		try {
-			pointsOut = new BufferedWriter(new FileWriter(new File(tmpDir+"/"+POINTS_FILE)));
-			StringBuilder buffer = new StringBuilder();
-
-			// DIMENSIONALITY + 1 means that the number of x(dimensionality) and target y
-			double[] point = new double[DIMENSIONALITY+1];
-
-			for (int i = 1; i <= numDataPoints; i++) {
-				point[0] = random.nextGaussian();
-				point[1] = 2 * point[0] + 0.01*random.nextGaussian();
-				writePoint(point, buffer, pointsOut);
-			}
-
-		}
-		finally {
-			if (pointsOut != null) {
-				pointsOut.close();
-			}
-		}
-
-		System.out.println("Wrote "+numDataPoints+" data points to "+tmpDir+"/"+POINTS_FILE);
-	}
-
-
-	private static void writePoint(double[] data, StringBuilder buffer, BufferedWriter out) throws IOException {
-		buffer.setLength(0);
-
-		// write coordinates
-		for (int j = 0; j < data.length; j++) {
-			buffer.append(FORMAT.format(data[j]));
-			if(j < data.length - 1) {
-				buffer.append(DELIMITER);
-			}
-		}
-
-		out.write(buffer.toString());
-		out.newLine();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
deleted file mode 100644
index a379bf8..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/EmptyFieldsCountAccumulator.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.example.java.relational;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-
-/**
- * This program filters lines from a CSV file with empty fields. In doing so, it counts the number of empty fields per
- * column within a CSV file using a custom accumulator for vectors. In this context, empty fields are those, that at
- * most contain whitespace characters like space and tab.
- * <p>
- * The input file is a plain text CSV file with the semicolon as field separator and double quotes as field delimiters
- * and three columns. See {@link #getDataSet(ExecutionEnvironment)} for configuration.
- * <p>
- * Usage: <code>FilterAndCountIncompleteLines [&lt;input file path&gt; [&lt;result path&gt;]]</code> <br>
- * <p>
- * This example shows how to use:
- * <ul>
- * <li>custom accumulators
- * <li>tuple data types
- * <li>inline-defined functions
- * </ul>
- */
-@SuppressWarnings("serial")
-public class EmptyFieldsCountAccumulator {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	private static final String EMPTY_FIELD_ACCUMULATOR = "empty-fields";
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// get the data set
-		final DataSet<Tuple> file = getDataSet(env);
-
-		// filter lines with empty fields
-		final DataSet<Tuple> filteredLines = file.filter(new EmptyFieldFilter());
-
-		// Here, we could do further processing with the filtered lines...
-		
-		// output the filtered lines
-		if (outputPath == null) {
-			filteredLines.print();
-		} else {
-			filteredLines.writeAsCsv(outputPath);
-		}
-
-		// execute program
-		final JobExecutionResult result = env.execute("Accumulator example");
-
-		// get the accumulator result via its registration key
-		final List<Integer> emptyFields = result.getAccumulatorResult(EMPTY_FIELD_ACCUMULATOR);
-		System.out.format("Number of detected empty fields per column: %s\n", emptyFields);
-
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static String filePath;
-	private static String outputPath;
-
-	private static boolean parseParameters(final String[] programArguments) {
-
-		if (programArguments.length >= 3) {
-			System.err.println("Usage: FilterAndCountIncompleteLines [<input file path> [<result path>]]");
-			return false;
-		}
-
-		if (programArguments.length >= 1) {
-			filePath = programArguments[0];
-			if (programArguments.length == 2) {
-				outputPath = programArguments[1];
-			}
-		}
-
-		return true;
-	}
-
-	@SuppressWarnings("unchecked")
-	private static DataSet<Tuple> getDataSet(final ExecutionEnvironment env) {
-
-		DataSet<? extends Tuple> source;
-		if (filePath == null) {
-			source = env.fromCollection(getExampleInputTuples());
-
-		} else {
-			source = env
-					.readCsvFile(filePath)
-					.fieldDelimiter(';')
-					.types(String.class, String.class, String.class);
-
-		}
-
-		return (DataSet<Tuple>) source;
-	}
-
-	private static Collection<Tuple3<String, String, String>> getExampleInputTuples() {
-		Collection<Tuple3<String, String, String>> inputTuples = new ArrayList<Tuple3<String, String, String>>();
-		inputTuples.add(new Tuple3<String, String, String>("John", "Doe", "Foo Str."));
-		inputTuples.add(new Tuple3<String, String, String>("Joe", "Johnson", ""));
-		inputTuples.add(new Tuple3<String, String, String>(null, "Kate Morn", "Bar Blvd."));
-		inputTuples.add(new Tuple3<String, String, String>("Tim", "Rinny", ""));
-		inputTuples.add(new Tuple3<String, String, String>("Alicia", "Jackson", "  "));
-		return inputTuples;
-	}
-
-	/**
-	 * This function filters all incoming tuples that have one or more empty fields.
-	 * In doing so, it also counts the number of empty fields per attribute with an accumulator (registered under 
-	 * {@link EmptyFieldsCountAccumulator#EMPTY_FIELD_ACCUMULATOR}).
-	 */
-	public static final class EmptyFieldFilter extends RichFilterFunction<Tuple> {
-
-		// create a new accumulator in each filter function instance
-		// accumulators can be merged later on
-		private final VectorAccumulator emptyFieldCounter = new VectorAccumulator();
-
-		@Override
-		public void open(final Configuration parameters) throws Exception {
-			super.open(parameters);
-
-			// register the accumulator instance
-			getRuntimeContext().addAccumulator(EMPTY_FIELD_ACCUMULATOR,
-					this.emptyFieldCounter);
-		}
-
-		@Override
-		public boolean filter(final Tuple t) {
-			boolean containsEmptyFields = false;
-
-			// iterate over the tuple fields looking for empty ones
-			for (int pos = 0; pos < t.getArity(); pos++) {
-
-				final String field = t.getField(pos);
-				if (field == null || field.trim().isEmpty()) {
-					containsEmptyFields = true;
-
-					// if an empty field is encountered, update the
-					// accumulator
-					this.emptyFieldCounter.add(pos);
-				}
-			}
-
-			return !containsEmptyFields;
-		}
-	}
-
-	/**
-	 * This accumulator lets you increase vector components distributedly. The {@link #add(Integer)} method lets you
-	 * increase the <i>n</i>-th vector component by 1, whereat <i>n</i> is the methods parameter. The size of the vector
-	 * is automatically managed.
-	 */
-	public static class VectorAccumulator implements Accumulator<Integer, List<Integer>> {
-
-		/** Stores the accumulated vector components. */
-		private final List<Integer> resultVector = new ArrayList<Integer>();
-
-		/**
-		 * Increases the result vector component at the specified position by 1.
-		 */
-		@Override
-		public void add(final Integer position) {
-			updateResultVector(position, 1);
-		}
-
-		/**
-		 * Increases the result vector component at the specified position by the specified delta.
-		 */
-		private void updateResultVector(final int position, final int delta) {
-			// inflate the vector to contain the given position
-			while (this.resultVector.size() <= position) {
-				this.resultVector.add(0);
-			}
-
-			// increment the component value
-			final int component = this.resultVector.get(position);
-			this.resultVector.set(position, component + delta);
-		}
-
-		@Override
-		public List<Integer> getLocalValue() {
-			return this.resultVector;
-		}
-
-		@Override
-		public void resetLocal() {
-			// clear the result vector if the accumulator instance shall be reused
-			this.resultVector.clear();
-		}
-
-		@Override
-		public void merge(final Accumulator<Integer, List<Integer>> other) {
-			// merge two vector accumulators by adding their up their vector components
-			final List<Integer> otherVector = other.getLocalValue();
-			for (int index = 0; index < otherVector.size(); index++) {
-				updateResultVector(index, otherVector.get(index));
-			}
-		}
-
-		@Override
-		public void write(final DataOutputView out) throws IOException {
-			// binary serialization of the result vector:
-			// [number of components, component 0, component 1, ...]
-			out.writeInt(this.resultVector.size());
-			for (final Integer component : this.resultVector) {
-				out.writeInt(component);
-			}
-		}
-
-		@Override
-		public void read(final DataInputView in) throws IOException {
-			// binary deserialization of the result vector
-			final int size = in.readInt();
-			for (int numReadComponents = 0; numReadComponents < size; numReadComponents++) {
-				final int component = in.readInt();
-				this.resultVector.add(component);
-			}
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java
deleted file mode 100644
index 08a261c..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/RelationalQuery.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.relational;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * This program implements the following relational query on the TPC-H data set.
- * 
- * <p>
- * <code><pre>
- * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
- *   FROM orders, lineitem
- *   WHERE l_orderkey = o_orderkey
- *     AND o_orderstatus = "X"
- *     AND YEAR(o_orderdate) > Y
- *     AND o_orderpriority LIKE "Z%"
- *   GROUP BY l_orderkey, o_shippriority;
- * </pre></code>
- *        
- * <p>
- * Input files are plain text CSV files using the pipe character ('|') as field separator 
- * as generated by the TPC-H data generator which is available at <a href="http://www.tpc.org/tpch/">http://www.tpc.org/tpch/</a>.
- * 
- * <p>
- * Usage: <code>RelationalQuery &lt;orders-csv path&gt; &lt;lineitem-csv path&gt; &lt;result path&gt;</code><br>
- *  
- * <p>
- * This example shows how to use:
- * <ul>
- * <li> tuple data types
- * <li> inline-defined functions
- * <li> projection and join projection
- * <li> build-in aggregation functions
- * </ul>
- */
-@SuppressWarnings("serial")
-public class RelationalQuery {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	private static String STATUS_FILTER = "F";
-	private static int YEAR_FILTER = 1993;
-	private static String OPRIO_FILTER = "5";
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
-		DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
-
-		// get lineitem data set: (orderkey, extendedprice)
-		DataSet<Tuple2<Integer, Double>> lineitems = getLineitemDataSet(env);
-
-		// orders filtered by year: (orderkey, custkey)
-		DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
-				// filter orders
-				orders.filter(
-								new FilterFunction<Tuple5<Integer, String, String, String, Integer>>() {
-									@Override
-									public boolean filter(Tuple5<Integer, String, String, String, Integer> t) {
-										// status filter
-										if(!t.f1.equals(STATUS_FILTER)) {
-											return false;
-										// year filter
-										} else if(Integer.parseInt(t.f2.substring(0, 4)) <= YEAR_FILTER) {
-											return false;
-										// order priority filter
-										} else if(!t.f3.startsWith(OPRIO_FILTER)) {
-											return false;
-										}
-										return true;
-									}
-								})
-				// project fields out that are no longer required
-				.project(0,4).types(Integer.class, Integer.class);
-
-		// join orders with lineitems: (orderkey, shippriority, extendedprice)
-		DataSet<Tuple3<Integer, Integer, Double>> lineitemsOfOrders = 
-				ordersFilteredByYear.joinWithHuge(lineitems)
-									.where(0).equalTo(0)
-									.projectFirst(0,1).projectSecond(1)
-									.types(Integer.class, Integer.class, Double.class);
-
-		// extendedprice sums: (orderkey, shippriority, sum(extendedprice))
-		DataSet<Tuple3<Integer, Integer, Double>> priceSums =
-				// group by order and sum extendedprice
-				lineitemsOfOrders.groupBy(0,1).aggregate(Aggregations.SUM, 2);
-
-		// emit result
-		priceSums.writeAsCsv(outputPath);
-		
-		// execute program
-		env.execute("Relational Query Example");
-		
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static String ordersPath;
-	private static String lineitemPath;
-	private static String outputPath;
-	
-	private static boolean parseParameters(String[] programArguments) {
-		
-		if(programArguments.length > 0) {
-			if(programArguments.length == 3) {
-				ordersPath = programArguments[0];
-				lineitemPath = programArguments[1];
-				outputPath = programArguments[2];
-			} else {
-				System.err.println("Usage: RelationalQuery <orders-csv path> <lineitem-csv path> <result path>");
-				return false;
-			}
-		} else {
-			System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
-								"  Due to legal restrictions, we can not ship generated data.\n" +
-								"  You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + 
-								"  Usage: RelationalQuery <orders-csv path> <lineitem-csv path> <result path>");
-			return false;
-		}
-		return true;
-	}
-	
-	private static DataSet<Tuple5<Integer, String, String, String, Integer>> getOrdersDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(ordersPath)
-					.fieldDelimiter('|')
-					.includeFields("101011010")
-					.types(Integer.class, String.class, String.class, String.class, Integer.class);
-	}
-
-	private static DataSet<Tuple2<Integer, Double>> getLineitemDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(lineitemPath)
-					.fieldDelimiter('|')
-					.includeFields("1000010000000000")
-					.types(Integer.class, Double.class);
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java
deleted file mode 100644
index 1ff6583..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery10.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.relational;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * This program implements a modified version of the TPC-H query 10.
- * The original query can be found at
- * <a href="http://www.tpc.org/tpch/spec/tpch2.16.0.pdf">http://www.tpc.org/tpch/spec/tpch2.16.0.pdf</a> (page 45).
- * 
- * <p>
- * This program implements the following SQL equivalent:
- * 
- * <p>
- * <code><pre>
- * SELECT 
- *        c_custkey,
- *        c_name, 
- *        c_address,
- *        n_name, 
- *        c_acctbal
- *        SUM(l_extendedprice * (1 - l_discount)) AS revenue,  
- * FROM   
- *        customer, 
- *        orders, 
- *        lineitem, 
- *        nation 
- * WHERE 
- *        c_custkey = o_custkey 
- *        AND l_orderkey = o_orderkey 
- *        AND YEAR(o_orderdate) > '1990' 
- *        AND l_returnflag = 'R' 
- *        AND c_nationkey = n_nationkey 
- * GROUP BY 
- *        c_custkey, 
- *        c_name, 
- *        c_acctbal, 
- *        n_name, 
- *        c_address
- * </pre></code>
- *        
- * <p>
- * Compared to the original TPC-H query this version does not print 
- * c_phone and c_comment, only filters by years greater than 1990 instead of
- * a period of 3 months, and does not sort the result by revenue.
- * 
- * <p>
- * Input files are plain text CSV files using the pipe character ('|') as field separator 
- * as generated by the TPC-H data generator which is available at <a href="http://www.tpc.org/tpch/">http://www.tpc.org/tpch/</a>.
- * 
- * <p>
- * Usage: <code>TPCHQuery10 &lt;customer-csv path&gt; &lt;orders-csv path&gt; &lt;lineitem-csv path&gt; &lt;nation-csv path&gt; &lt;result path&gt;</code><br>
- *  
- * <p>
- * This example shows how to use:
- * <ul>
- * <li> tuple data types
- * <li> inline-defined functions
- * <li> projection and join projection
- * <li> build-in aggregation functions
- * </ul>
- */
-@SuppressWarnings("serial")
-public class TPCHQuery10 {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// get customer data set: (custkey, name, address, nationkey, acctbal) 
-		DataSet<Tuple5<Integer, String, String, Integer, Double>> customers = getCustomerDataSet(env);
-
-		// get orders data set: (orderkey, custkey, orderdate)
-		DataSet<Tuple3<Integer, Integer, String>> orders = getOrdersDataSet(env);
-
-		// get lineitem data set: (orderkey, extendedprice, discount, returnflag)
-		DataSet<Tuple4<Integer, Double, Double, String>> lineitems = getLineitemDataSet(env);
-
-		// get nation data set: (nationkey, name)
-		DataSet<Tuple2<Integer, String>> nations = getNationsDataSet(env);
-
-		// orders filtered by year: (orderkey, custkey)
-		DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
-				// filter by year
-				orders.filter(
-								new FilterFunction<Tuple3<Integer,Integer, String>>() {
-									@Override
-									public boolean filter(Tuple3<Integer, Integer, String> t) {
-										int year = Integer.parseInt(t.f2.substring(0, 4));
-										return year > 1990;
-									}
-								})
-				// project fields out that are no longer required
-				.project(0,1).types(Integer.class, Integer.class);
-
-		// lineitems filtered by flag: (orderkey, extendedprice, discount)
-		DataSet<Tuple3<Integer, Double, Double>> lineitemsFilteredByFlag = 
-				// filter by flag
-				lineitems.filter(new FilterFunction<Tuple4<Integer, Double, Double, String>>() {
-										@Override
-										public boolean filter(Tuple4<Integer, Double, Double, String> t)
-												throws Exception {
-											return t.f3.equals("R");
-										}
-								})
-				// project fields out that are no longer required
-				.project(0,1,2).types(Integer.class, Double.class, Double.class);
-
-		// join orders with lineitems: (custkey, extendedprice, discount)
-		DataSet<Tuple3<Integer, Double, Double>> lineitemsOfCustomerKey = 
-				ordersFilteredByYear.joinWithHuge(lineitemsFilteredByFlag)
-									.where(0).equalTo(0)
-									.projectFirst(1).projectSecond(1,2)
-									.types(Integer.class, Double.class, Double.class);
-
-		// aggregate for revenue: (custkey, revenue)
-		DataSet<Tuple2<Integer, Double>> revenueOfCustomerKey = lineitemsOfCustomerKey
-				// calculate the revenue for each item
-				.map(new MapFunction<Tuple3<Integer, Double, Double>, Tuple2<Integer, Double>>() {
-							@Override
-							public Tuple2<Integer, Double> map(Tuple3<Integer, Double, Double> t) {
-								// revenue per item = l_extendedprice * (1 - l_discount)
-								return new Tuple2<Integer, Double>(t.f0, t.f1 * (1 - t.f2));
-							}
-					})
-				// aggregate the revenues per item to revenue per customer
-				.groupBy(0).aggregate(Aggregations.SUM, 1);
-
-		// join customer with nation (custkey, name, address, nationname, acctbal)
-		DataSet<Tuple5<Integer, String, String, String, Double>> customerWithNation = customers
-						.joinWithTiny(nations)
-						.where(3).equalTo(0)
-						.projectFirst(0,1,2).projectSecond(1).projectFirst(4)
-						.types(Integer.class, String.class, String.class, String.class, Double.class);
-
-		// join customer (with nation) with revenue (custkey, name, address, nationname, acctbal, revenue)
-		DataSet<Tuple6<Integer, String, String, String, Double, Double>> customerWithRevenue = 
-				customerWithNation.join(revenueOfCustomerKey)
-				.where(0).equalTo(0)
-				.projectFirst(0,1,2,3,4).projectSecond(1)
-				.types(Integer.class, String.class, String.class, String.class, Double.class, Double.class);
-
-		// emit result
-		customerWithRevenue.writeAsCsv(outputPath);
-		
-		// execute program
-		env.execute("TPCH Query 10 Example");
-		
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static String customerPath;
-	private static String ordersPath;
-	private static String lineitemPath;
-	private static String nationPath;
-	private static String outputPath;
-	
-	private static boolean parseParameters(String[] programArguments) {
-		
-		if(programArguments.length > 0) {
-			if(programArguments.length == 5) {
-				customerPath = programArguments[0];
-				ordersPath = programArguments[1];
-				lineitemPath = programArguments[2];
-				nationPath = programArguments[3];
-				outputPath = programArguments[4];
-			} else {
-				System.err.println("Usage: TPCHQuery10 <customer-csv path> <orders-csv path> <lineitem-csv path> <nation-csv path> <result path>");
-				return false;
-			}
-		} else {
-			System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
-								"  Due to legal restrictions, we can not ship generated data.\n" +
-								"  You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + 
-								"  Usage: TPCHQuery10 <customer-csv path> <orders-csv path> <lineitem-csv path> <nation-csv path> <result path>");
-			return false;
-		}
-		return true;
-	}
-	
-	private static DataSet<Tuple5<Integer, String, String, Integer, Double>> getCustomerDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(customerPath)
-					.fieldDelimiter('|')
-					.includeFields("11110100")
-					.types(Integer.class, String.class, String.class, Integer.class, Double.class);
-	}
-	
-	private static DataSet<Tuple3<Integer, Integer, String>> getOrdersDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(ordersPath)
-					.fieldDelimiter('|')
-					.includeFields("110010000")
-					.types(Integer.class, Integer.class, String.class);
-	}
-
-	private static DataSet<Tuple4<Integer, Double, Double, String>> getLineitemDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(lineitemPath)
-					.fieldDelimiter('|')
-					.includeFields("1000011010000000")
-					.types(Integer.class, Double.class, Double.class, String.class);
-	}
-	
-	private static DataSet<Tuple2<Integer, String>> getNationsDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(nationPath)
-					.fieldDelimiter('|')
-					.includeFields("1100")
-					.types(Integer.class, String.class);
-	}
-			
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java
deleted file mode 100644
index 4544fd4..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/TPCHQuery3.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.example.java.relational;
-
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.Date;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * This program implements a modified version of the TPC-H query 3. The
- * example demonstrates how to assign names to fields by extending the Tuple class.
- * The original query can be found at
- * <a href="http://www.tpc.org/tpch/spec/tpch2.16.0.pdf">http://www.tpc.org/tpch/spec/tpch2.16.0.pdf</a> (page 29).
- *
- * <p>
- * This program implements the following SQL equivalent:
- *
- * <p>
- * <code><pre>
- * SELECT 
- *      l_orderkey, 
- *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
- *      o_orderdate, 
- *      o_shippriority 
- * FROM customer, 
- *      orders, 
- *      lineitem 
- * WHERE
- *      c_mktsegment = '[SEGMENT]' 
- *      AND c_custkey = o_custkey
- *      AND l_orderkey = o_orderkey
- *      AND o_orderdate < date '[DATE]'
- *      AND l_shipdate > date '[DATE]'
- * GROUP BY
- *      l_orderkey, 
- *      o_orderdate, 
- *      o_shippriority;
- * </pre></code>
- *
- * <p>
- * Compared to the original TPC-H query this version does not sort the result by revenue
- * and orderdate.
- *
- * <p>
- * Input files are plain text CSV files using the pipe character ('|') as field separator 
- * as generated by the TPC-H data generator which is available at <a href="http://www.tpc.org/tpch/">http://www.tpc.org/tpch/</a>.
- *
- *  <p>
- * Usage: <code>TPCHQuery3 &lt;lineitem-csv path&gt; &lt;customer-csv path&gt; &lt;orders-csv path&gt; &lt;result path&gt;</code><br>
- *  
- * <p>
- * This example shows how to use:
- * <ul>
- * <li> custom data type derived from tuple data types
- * <li> inline-defined functions
- * <li> build-in aggregation functions
- * </ul>
- */
-@SuppressWarnings("serial")
-public class TPCHQuery3 {
-
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		DataSet<Lineitem> li = getLineitemDataSet(env);
-		DataSet<Order> or = getOrdersDataSet(env);
-		DataSet<Customer> cust = getCustomerDataSet(env);
-		
-		// Filter market segment "AUTOMOBILE"
-		cust = cust.filter(
-							new FilterFunction<Customer>() {
-								@Override
-								public boolean filter(Customer value) {
-									return value.getMktsegment().equals("AUTOMOBILE");
-								}
-							});
-
-		// Filter all Orders with o_orderdate < 12.03.1995
-		or = or.filter(
-						new FilterFunction<Order>() {
-							private DateFormat format = new SimpleDateFormat("yyyy-MM-dd");
-							private Date date;
-							
-							{	
-								Calendar cal = Calendar.getInstance();
-								cal.set(1995, 3, 12);
-								date = cal.getTime(); 
-							}
-							
-							@Override
-							public boolean filter(Order value) throws ParseException {
-								Date orderDate = format.parse(value.getOrderdate());
-								return orderDate.before(date);
-							}
-						});
-		
-		// Filter all Lineitems with l_shipdate > 12.03.1995
-		li = li.filter(
-						new FilterFunction<Lineitem>() {
-							private DateFormat format = new SimpleDateFormat("yyyy-MM-dd");
-							private Date date;
-							
-							{
-								Calendar cal = Calendar.getInstance();
-								cal.set(1995, 3, 12);
-								date = cal.getTime();
-							}
-							
-							@Override
-							public boolean filter(Lineitem value) throws ParseException {
-								Date shipDate = format.parse(value.getShipdate());
-								return shipDate.after(date);
-							}
-						});
-
-		// Join customers with orders and package them into a ShippingPriorityItem
-		DataSet<ShippingPriorityItem> customerWithOrders = 
-				cust.join(or)
-					.where(0)
-					.equalTo(0)
-					.with(
-							new JoinFunction<Customer, Order, ShippingPriorityItem>() {
-								@Override
-								public ShippingPriorityItem join(Customer first, Order second) {
-									return new ShippingPriorityItem(0, 0.0, second.getOrderdate(),
-											second.getShippriority(), second.getOrderkey());
-								}
-							});
-		
-		// Join the last join result with Lineitems
-		DataSet<ShippingPriorityItem> joined = 
-				customerWithOrders.join(li)
-									.where(4)
-									.equalTo(0)
-									.with(
-											new JoinFunction<ShippingPriorityItem, Lineitem, ShippingPriorityItem>() {
-												@Override
-												public ShippingPriorityItem join(ShippingPriorityItem first, Lineitem second) {
-													first.setL_Orderkey(second.getOrderkey());
-													first.setRevenue(second.getExtendedprice() * (1 - second.getDiscount()));
-													return first;
-												}
-											});
-		
-		// Group by l_orderkey, o_orderdate and o_shippriority and compute revenue sum
-		joined = joined
-				.groupBy(0, 2, 3)
-				.aggregate(Aggregations.SUM, 1);
-		
-		// emit result
-		joined.writeAsCsv(outputPath, "\n", "|");
-		
-		// execute program
-		env.execute("TPCH Query 3 Example");
-		
-	}
-	
-	// *************************************************************************
-	//     DATA TYPES
-	// *************************************************************************
-	
-	public static class Lineitem extends Tuple4<Integer, Double, Double, String> {
-
-		public Integer getOrderkey() { return this.f0; }
-		public Double getDiscount() { return this.f2; }
-		public Double getExtendedprice() { return this.f1; }
-		public String getShipdate() { return this.f3; }
-	}
-
-	public static class Customer extends Tuple2<Integer, String> {
-		
-		public Integer getCustKey() { return this.f0; }
-		public String getMktsegment() { return this.f1; }
-	}
-
-	public static class Order extends Tuple3<Integer, String, Integer> {
-		
-		public Integer getOrderkey() { return this.f0; }
-		public String getOrderdate() { return this.f1; }
-		public Integer getShippriority() { return this.f2; }
-	}
-
-	public static class ShippingPriorityItem extends Tuple5<Integer, Double, String, Integer, Integer> {
-
-		public ShippingPriorityItem() { }
-
-		public ShippingPriorityItem(Integer l_orderkey, Double revenue,
-				String o_orderdate, Integer o_shippriority, Integer o_orderkey) {
-			this.f0 = l_orderkey;
-			this.f1 = revenue;
-			this.f2 = o_orderdate;
-			this.f3 = o_shippriority;
-			this.f4 = o_orderkey;
-		}
-		
-		public Integer getL_Orderkey() { return this.f0; }
-		public void setL_Orderkey(Integer l_orderkey) { this.f0 = l_orderkey; }
-		public Double getRevenue() { return this.f1; }
-		public void setRevenue(Double revenue) { this.f1 = revenue; }
-		
-		public String getOrderdate() { return this.f2; }
-		public Integer getShippriority() { return this.f3; }
-		public Integer getO_Orderkey() { return this.f4; }
-	}
-	
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static String lineitemPath;
-	private static String customerPath;
-	private static String ordersPath;
-	private static String outputPath;
-	
-	private static boolean parseParameters(String[] programArguments) {
-		
-		if(programArguments.length > 0) {
-			if(programArguments.length == 4) {
-				lineitemPath = programArguments[0];
-				customerPath = programArguments[1];
-				ordersPath = programArguments[2];
-				outputPath = programArguments[3];
-			} else {
-				System.err.println("Usage: TPCHQuery3 <lineitem-csv path> <customer-csv path> <orders-csv path> <result path>");
-				return false;
-			}
-		} else {
-			System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
-								"  Due to legal restrictions, we can not ship generated data.\n" +
-								"  You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" + 
-								"  Usage: TPCHQuery3 <lineitem-csv path> <customer-csv path> <orders-csv path> <result path>");
-			return false;
-		}
-		return true;
-	}
-	
-	private static DataSet<Lineitem> getLineitemDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(lineitemPath)
-					.fieldDelimiter('|')
-					.includeFields("1000011000100000")
-					.tupleType(Lineitem.class);
-	}
-	
-	private static DataSet<Customer> getCustomerDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(customerPath)
-					.fieldDelimiter('|')
-					.includeFields("10000010")
-					.tupleType(Customer.class);
-	}
-	
-	private static DataSet<Order> getOrdersDataSet(ExecutionEnvironment env) {
-		return env.readCsvFile(ordersPath)
-					.fieldDelimiter('|')
-					.includeFields("100010010")
-					.tupleType(Order.class);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cce46eb/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java
deleted file mode 100644
index 9ca6ea9..0000000
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/relational/WebLogAnalysis.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.example.java.relational;
-
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.example.java.relational.util.WebLogData;
-import org.apache.flink.example.java.relational.util.WebLogDataGenerator;
-
-/**
- * This program processes web logs and relational data. 
- * It implements the following relational query:
- *
- * <code><pre>
- * SELECT 
- *       r.pageURL, 
- *       r.pageRank, 
- *       r.avgDuration
- * FROM documents d JOIN rankings r
- *                  ON d.url = r.url
- * WHERE CONTAINS(d.text, [keywords]) 
- *       AND r.rank > [rank] 
- *       AND NOT EXISTS 
- *           (
- *              SELECT * FROM Visits v
- *              WHERE v.destUrl = d.url 
- *                    AND v.visitDate < [date]
- *           );
- * </pre></code>
- *
- * <p>
- * Input files are plain text CSV files using the pipe character ('|') as field separator.
- * The tables referenced in the query can be generated using the {@link WebLogDataGenerator} and 
- * have the following schemas
- * <code><pre>
- * CREATE TABLE Documents (
- *                url VARCHAR(100) PRIMARY KEY,
- *                contents TEXT );
- *
- * CREATE TABLE Rankings (
- *                pageRank INT,
- *                pageURL VARCHAR(100) PRIMARY KEY,
- *                avgDuration INT );
- *
- * CREATE TABLE Visits (
- *                sourceIP VARCHAR(16),
- *                destURL VARCHAR(100),
- *                visitDate DATE,
- *                adRevenue FLOAT,
- *                userAgent VARCHAR(64),
- *                countryCode VARCHAR(3),
- *                languageCode VARCHAR(6),
- *                searchWord VARCHAR(32),
- *                duration INT );
- * </pre></code>
- * 
- * <p>
- * Usage: <code>WebLogAnalysis &lt;documents path&gt; &lt;ranks path&gt; &lt;visits path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WebLogData}.
- * 
- * <p>
- * This example shows how to use:
- * <ul>
- * <li> tuple data types
- * <li> projection and join projection
- * <li> the CoGroup transformation for an anti-join
- * </ul>
- * 
- */
-@SuppressWarnings("serial")
-public class WebLogAnalysis {
-	
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-	
-	public static void main(String[] args) throws Exception {
-		
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		DataSet<Tuple2<String, String>> documents = getDocumentsDataSet(env);
-		DataSet<Tuple3<Integer, String, Integer>> ranks = getRanksDataSet(env);
-		DataSet<Tuple2<String, String>> visits = getVisitsDataSet(env);
-		
-		// Retain documents with keywords
-		DataSet<Tuple1<String>> filterDocs = documents
-				.filter(new FilterDocByKeyWords())
-				.project(0).types(String.class);
-
-		// Filter ranks by minimum rank
-		DataSet<Tuple3<Integer, String, Integer>> filterRanks = ranks
-				.filter(new FilterByRank());
-
-		// Filter visits by visit date
-		DataSet<Tuple1<String>> filterVisits = visits
-				.filter(new FilterVisitsByDate())
-				.project(0).types(String.class);
-
-		// Join the filtered documents and ranks, i.e., get all URLs with min rank and keywords
-		DataSet<Tuple3<Integer, String, Integer>> joinDocsRanks = 
-				filterDocs.join(filterRanks)
-							.where(0).equalTo(1)
-							.projectSecond(0,1,2)
-							.types(Integer.class, String.class, Integer.class);
-
-		// Anti-join urls with visits, i.e., retain all URLs which have NOT been visited in a certain time
-		DataSet<Tuple3<Integer, String, Integer>> result = 
-				joinDocsRanks.coGroup(filterVisits)
-								.where(1).equalTo(0)
-								.with(new AntiJoinVisits());
-
-		// emit result
-		if(fileOutput) {
-			result.writeAsCsv(outputPath, "\n", "|");
-		} else {
-			result.print();
-		}
-
-		// execute program
-		env.execute("WebLogAnalysis Example");
-		
-	}
-	
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-	
-	/**
-	 * MapFunction that filters for documents that contain a certain set of
-	 * keywords.
-	 */
-	public static class FilterDocByKeyWords implements FilterFunction<Tuple2<String, String>> {
-
-		private static final String[] KEYWORDS = { " editors ", " oscillations " };
-
-		/**
-		 * Filters for documents that contain all of the given keywords and projects the records on the URL field.
-		 *
-		 * Output Format:
-		 * 0: URL
-		 * 1: DOCUMENT_TEXT
-		 */
-		@Override
-		public boolean filter(Tuple2<String, String> value) throws Exception {
-			// FILTER
-			// Only collect the document if all keywords are contained
-			String docText = value.f1;
-			for (String kw : KEYWORDS) {
-				if (!docText.contains(kw)) {
-					return false;
-				}
-			}
-			return true;
-		}
-	}
-
-	/**
-	 * MapFunction that filters for records where the rank exceeds a certain threshold.
-	 */
-	public static class FilterByRank implements FilterFunction<Tuple3<Integer, String, Integer>> {
-
-		private static final int RANKFILTER = 40;
-
-		/**
-		 * Filters for records of the rank relation where the rank is greater
-		 * than the given threshold.
-		 *
-		 * Output Format:
-		 * 0: RANK
-		 * 1: URL
-		 * 2: AVG_DURATION
-		 */
-		@Override
-		public boolean filter(Tuple3<Integer, String, Integer> value) throws Exception {
-			return (value.f0 > RANKFILTER);
-		}
-	}
-
-	/**
-	 * MapFunction that filters for records of the visits relation where the year
-	 * (from the date string) is equal to a certain value.
-	 */
-	public static class FilterVisitsByDate implements FilterFunction<Tuple2<String, String>> {
-
-		private static final int YEARFILTER = 2007;
-
-		/**
-		 * Filters for records of the visits relation where the year of visit is equal to a
-		 * specified value. The URL of all visit records passing the filter is emitted.
-		 *
-		 * Output Format:
-		 * 0: URL
-		 * 1: DATE
-		 */
-		@Override
-		public boolean filter(Tuple2<String, String> value) throws Exception {
-			// Parse date string with the format YYYY-MM-DD and extract the year
-			String dateString = value.f1;
-			int year = Integer.parseInt(dateString.substring(0,4));
-			return (year == YEARFILTER);
-		}
-	}
-
-
-	/**
-	 * CoGroupFunction that realizes an anti-join.
-	 * If the first input does not provide any pairs, all pairs of the second input are emitted.
-	 * Otherwise, no pair is emitted.
-	 */
-	public static class AntiJoinVisits implements CoGroupFunction<Tuple3<Integer, String, Integer>, Tuple1<String>, Tuple3<Integer, String, Integer>> {
-
-		/**
-		 * If the visit iterator is empty, all pairs of the rank iterator are emitted.
-		 * Otherwise, no pair is emitted.
-		 *
-		 * Output Format:
-		 * 0: RANK
-		 * 1: URL
-		 * 2: AVG_DURATION
-		 */
-		@Override
-		public void coGroup(Iterable<Tuple3<Integer, String, Integer>> ranks, Iterable<Tuple1<String>> visits, Collector<Tuple3<Integer, String, Integer>> out) {
-			// Check if there is a entry in the visits relation
-			if (!visits.iterator().hasNext()) {
-				for (Tuple3<Integer, String, Integer> next : ranks) {
-					// Emit all rank pairs
-					out.collect(next);
-				}
-			}
-		}
-	}
-
-	// *************************************************************************
-	//     UTIL METHODS
-	// *************************************************************************
-	
-	private static boolean fileOutput = false;
-	private static String documentsPath;
-	private static String ranksPath;
-	private static String visitsPath;
-	private static String outputPath;
-	
-	private static boolean parseParameters(String[] args) {
-		
-		if(args.length > 0) {
-			fileOutput = true;
-			if(args.length == 4) {
-				documentsPath = args[0];
-				ranksPath = args[1];
-				visitsPath = args[2];
-				outputPath = args[3];
-			} else {
-				System.err.println("Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing WebLog Analysis example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  We provide a data generator to create synthetic input files for this program.");
-			System.out.println("  Usage: WebLogAnalysis <documents path> <ranks path> <visits path> <result path>");
-		}
-		return true;
-	}
-	
-	private static DataSet<Tuple2<String, String>> getDocumentsDataSet(ExecutionEnvironment env) {
-		// Create DataSet for documents relation (URL, Doc-Text)
-		if(fileOutput) {
-			return env.readCsvFile(documentsPath)
-						.fieldDelimiter('|')
-						.types(String.class, String.class);
-		} else {
-			return WebLogData.getDocumentDataSet(env);
-		}
-	}
-	
-	private static DataSet<Tuple3<Integer, String, Integer>> getRanksDataSet(ExecutionEnvironment env) {
-		// Create DataSet for ranks relation (Rank, URL, Avg-Visit-Duration)
-		if(fileOutput) {
-			return env.readCsvFile(ranksPath)
-						.fieldDelimiter('|')
-						.types(Integer.class, String.class, Integer.class);
-		} else {
-			return WebLogData.getRankDataSet(env);
-		}
-	}
-
-	private static DataSet<Tuple2<String, String>> getVisitsDataSet(ExecutionEnvironment env) {
-		// Create DataSet for visits relation (URL, Date)
-		if(fileOutput) {
-			return env.readCsvFile(visitsPath)
-						.fieldDelimiter('|')
-						.includeFields("011000000")
-						.types(String.class, String.class);
-		} else {
-			return WebLogData.getVisitDataSet(env);
-		}
-	}
-		
-}


[41/60] git commit: [scala] Fix Formatting in Examples and add ITCases

Posted by al...@apache.org.
[scala] Fix Formatting in Examples and add ITCases

Also actually use termination criterion in TransitivelClosureNaive
Java example.

Add ConnectedComponentsITCase for Scala Example

Also fix some formatting in the example code

Add WebLogAnalysisITCase for Scala Example

Some minor reformatting of example code and scaladoc.

Add ITCases for TriangleEnumeration Scala Examples

Also fix some formatting and make TriangleEnumerationOpt Scala produce the
same output as the Java version.

Add PageRankITCase for Scala Example

Also fix formatting in PageRank Scala Example.

Fix formatting in EnumTriangles Scala Examples

Remove Old/Deprecated Scala Examples and ITCases

Fix formatting in EnumTrianglesBasic.scala

Fix formatting in LinearRegression Scala Example

Remove old Scala LineRank Code and RelQuery Example

[scala] Fix typo in scaladoc in GroupedDataSet

[scala] Fix Scaladoc of Join and CoGroup Operation

Was still referring to the type of join/coGroup function that returns an
Option.

Fix tab vs. spaces in flink-scala and flink-scala-examples


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

Branch: refs/heads/master
Commit: 31ed0c4ccd1103d67d8e9331399f963764f38e33
Parents: a41a29b
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Sep 10 17:59:55 2014 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 22 09:59:59 2014 +0200

----------------------------------------------------------------------
 .../java/graph/TransitiveClosureNaive.java      |   2 +-
 .../graph/util/ConnectedComponentsData.java     |  12 +-
 flink-examples/flink-scala-examples/pom.xml     |  54 +++---
 .../scala/graph/EnumTrianglesBasic.scala        |  43 +++--
 .../examples/scala/graph/EnumTrianglesOpt.scala |   2 +
 .../flink/examples/scala/graph/LineRank.scala   |  96 ----------
 .../scala/graph/TransitiveClosureNaive.scala    |   2 +-
 .../scala/iterative/TerminationCriterion.scala  |  78 ---------
 .../examples/scala/ml/LinearRegression.scala    | 175 +++++++------------
 .../scala/relational/RelationalQuery.scala      | 107 ------------
 .../examples/scala/testing/KMeansForTest.scala  | 105 -----------
 .../scala/wordcount/WordCountWithCount.scala    |  63 -------
 .../WordCountWithUserDefinedType.scala          |  59 -------
 flink-scala/pom.xml                             |  38 ++--
 .../org/apache/flink/api/scala/DataSet.scala    |  17 +-
 .../apache/flink/api/scala/GroupedDataSet.scala |   4 +-
 .../ComputeEdgeDegreesITCase.java               |  39 -----
 .../ConnectedComponentsITCase.java              |  94 ++++++----
 .../EnumTriangleBasicITCase.java                |  46 +++++
 .../EnumTriangleOptITCase.java                  |  46 +++++
 .../EnumTrianglesOnEdgesWithDegreesITCase.java  |  39 -----
 .../exampleScalaPrograms/PageRankITCase.java    |  99 +++++++++++
 .../RelationalQueryITCase.java                  |  49 ------
 .../TransitiveClosureNaiveITCase.java           |  54 ------
 .../WebLogAnalysisITCase.java                   |  81 +++++----
 .../WordCountPactValueITCase.java               |  33 ----
 .../WordCountWithCountFunctionITCase.java       |  31 ----
 27 files changed, 447 insertions(+), 1021 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
index 30230d6..9754a34 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java
@@ -94,7 +94,7 @@ public class TransitiveClosureNaive implements ProgramDescription {
 					}
 				});
 
-		DataSet<Tuple2<Long, Long>> transitiveClosure = paths.closeWith(nextPaths);
+		DataSet<Tuple2<Long, Long>> transitiveClosure = paths.closeWith(nextPaths, newPaths);
 
 
 		// emit result

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
index 27c7d45..59df7fe 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java
@@ -33,17 +33,13 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  */
 public class ConnectedComponentsData {
 	
-	public static final Object[][] VERTICES  = new Object[][] {
-		new Object[]{1L}, new Object[]{2L}, new Object[]{3L}, new Object[]{4L}, 
-		new Object[]{5L},new Object[]{6L}, new Object[]{7L}, new Object[]{8L}, 
-		new Object[]{9L}, new Object[]{10L}, new Object[]{11L}, new Object[]{12L}, 
-		new Object[]{13L}, new Object[]{14L}, new Object[]{15L}, new Object[]{16L}
-	};
+	public static final long[] VERTICES  = new long[] {
+			1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16};
 
 	public static DataSet<Long> getDefaultVertexDataSet(ExecutionEnvironment env) {
 		List<Long> verticesList = new LinkedList<Long>();
-		for (Object[] vertex : VERTICES) {
-			verticesList.add((Long) vertex[0]);
+		for (long vertexId : VERTICES) {
+			verticesList.add(vertexId);
 		}
 		return env.fromCollection(verticesList);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/pom.xml b/flink-examples/flink-scala-examples/pom.xml
index 3b50c8b..79053d6 100644
--- a/flink-examples/flink-scala-examples/pom.xml
+++ b/flink-examples/flink-scala-examples/pom.xml
@@ -38,11 +38,11 @@ under the License.
 			<artifactId>flink-scala</artifactId>
 			<version>${project.version}</version>
 		</dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-java-examples</artifactId>
-            <version>${project.version}</version>
-        </dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java-examples</artifactId>
+			<version>${project.version}</version>
+		</dependency>
 	</dependencies>
 
 	<build>
@@ -54,7 +54,7 @@ under the License.
 				<version>3.1.4</version>
 				<executions>
 					<!-- Run scala compiler in the process-resources phase, so that dependencies on
-					    scala classes can be resolved later in the (Java) compile phase -->
+						scala classes can be resolved later in the (Java) compile phase -->
 					<execution>
 						<id>scala-compile-first</id>
 						<phase>process-resources</phase>
@@ -64,7 +64,7 @@ under the License.
 					</execution>
  
 					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
-					     scala classes can be resolved later in the (Java) test-compile phase -->
+						 scala classes can be resolved later in the (Java) test-compile phase -->
 					<execution>
 						<id>scala-test-compile</id>
 						<phase>process-test-resources</phase>
@@ -237,7 +237,7 @@ under the License.
 							</includes>
 						</configuration>
 					</execution>
-		               -->
+					   -->
 
 					<execution>
 						<id>WordCount</id>
@@ -260,7 +260,7 @@ under the License.
 							</includes>
 						</configuration>
 					</execution>
-                    <!--
+					<!--
 					<execution>
 						<id>ConnectedComponents</id>
 						<phase>package</phase>
@@ -282,27 +282,27 @@ under the License.
 					
 					-->
 
-                    <execution>
-                        <id>TransitiveClosureNaive</id>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>jar</goal>
-                        </goals>
+					<execution>
+						<id>TransitiveClosureNaive</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
 
-                        <configuration>
-                            <classifier>TransitiveClosureNaive</classifier>
+						<configuration>
+							<classifier>TransitiveClosureNaive</classifier>
 
-                            <archive>
-                                <manifestEntries>
-                                    <program-class>org.apache.flink.examples.scala.graph.TransitiveClosureNaive</program-class>
-                                </manifestEntries>
-                            </archive>
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.examples.scala.graph.TransitiveClosureNaive</program-class>
+								</manifestEntries>
+							</archive>
 
-                            <includes>
-                                <include>**/wordcount/TransitiveClosureNaive*.class</include>
-                            </includes>
-                        </configuration>
-                    </execution>
+							<includes>
+								<include>**/wordcount/TransitiveClosureNaive*.class</include>
+							</includes>
+						</configuration>
+					</execution>
 
 				</executions>
 			</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
index c920c31..fe121d5 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesBasic.scala
@@ -24,27 +24,26 @@ import org.apache.flink.api.common.functions.GroupReduceFunction
 import org.apache.flink.util.Collector
 import org.apache.flink.examples.java.graph.util.EnumTrianglesData
 import org.apache.flink.api.common.operators.Order
-import scala.collection.mutable.MutableList
+
+import scala.collection.mutable
 
 
 /**
  * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.
  * A triangle consists of three edges that connect three vertices with each other.
  * 
- * <p>
- * The algorithm works as follows: 
+ * The algorithm works as follows:
  * It groups all edges that share a common vertex and builds triads, i.e., triples of vertices 
  * that are connected by two edges. Finally, all triads are filtered for which no third edge exists 
  * that closes the triangle.
  *  
- * <p>
  * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Edges are represented as pairs for vertex IDs which are separated by space 
- * characters. Edges are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63\n"</code> gives four (undirected) edges (1)-(2), (2)-(12), (1)-(12), and (42)-(63)
- * that include a triangle
- * </ul>
+ *
+ *  - Edges are represented as pairs for vertex IDs which are separated by space
+ *   characters. Edges are separated by new-line characters.
+ *   For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) edges (1)-(2), (2)-(12),
+ *   (1)-(12), and (42)-(63) that include a triangle
+ *
  * <pre>
  *     (1)
  *     /  \
@@ -59,13 +58,11 @@ import scala.collection.mutable.MutableList
  * If no parameters are provided, the program is run with default data from 
  * [[org.apache.flink.examples.java.graph.util.EnumTrianglesData]]
  * 
- * <p>
  * This example shows how to use:
- * <ul>
- * <li>Custom Java objects which extend Tuple
- * <li>Group Sorting
- * </ul>
- * 
+ *
+ *  - Custom Java objects which extend Tuple
+ *  - Group Sorting
+ *
  */
 object EnumTrianglesBasic {
 	
@@ -91,7 +88,7 @@ object EnumTrianglesBasic {
 		
 		// emit result
 		if (fileOutput) {
-			triangles.writeAsCsv(outputPath, "\n", " ")
+			triangles.writeAsCsv(outputPath, "\n", ",")
 		} else {
 			triangles.print()
 		}
@@ -119,12 +116,12 @@ object EnumTrianglesBasic {
 	 */
 	class TriadBuilder extends GroupReduceFunction[Edge, Triad] {
 
-		val vertices = MutableList[Integer]()
+		val vertices = mutable.MutableList[Integer]()
 		
 		override def reduce(edges: java.lang.Iterable[Edge], out: Collector[Triad]) = {
 			
 			// clear vertex list
-			vertices.clear
+			vertices.clear()
 
 			// build and emit triads
 			for(e <- edges.asScala) {
@@ -153,10 +150,10 @@ object EnumTrianglesBasic {
 				false
 			}
 		} else {
-			System.out.println("Executing Enum Triangles Basic example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>");
+			System.out.println("Executing Enum Triangles Basic example with built-in default data.")
+			System.out.println("  Provide parameters to read input data from files.")
+			System.out.println("  See the documentation for the correct format of input files.")
+			System.out.println("  Usage: EnumTriangleBasic <edge path> <result path>")
 		}
 		true
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
index 80cce35..9370491 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/EnumTrianglesOpt.scala
@@ -26,6 +26,8 @@ import org.apache.flink.examples.java.graph.util.EnumTrianglesData
 import org.apache.flink.api.common.operators.Order
 import scala.collection.mutable.MutableList
 
+import scala.collection.mutable
+
 
 /**
  * Triangle enumeration is a pre-processing step to find closely connected parts in graphs.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
deleted file mode 100644
index 6902a6f..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/LineRank.scala
+++ /dev/null
@@ -1,96 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.examples.scala.graph
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.{ ProgramDescription, Program }
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//
-//class LineRank extends Program with Serializable {
-//
-//  case class Edge(source: Int, target: Int, weight: Double)
-//  case class VectorElement(index: Int, value: Double)
-//
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1), args(2), args(3).toInt, args(4))
-//  }
-//
-//  def sumElements(elem1: VectorElement, elem2: VectorElement) = VectorElement(elem1.index, elem1.value + elem2.value)
-//
-//  def sgtTimes(SGT: DataSetOLD[Edge], vector: DataSetOLD[VectorElement]) = {
-//    SGT.join(vector).where(_.source).isEqualTo(_.index)
-//      .map((edge, elem) => VectorElement(edge.target, edge.weight * elem.value))
-//      .groupBy(_.index).reduce(sumElements)
-//  }
-//
-//  def tgTimes(TG: DataSetOLD[Edge], vector: DataSetOLD[VectorElement]) = {
-//    TG.join(vector).where(_.target).isEqualTo(_.index)
-//      .map((edge, elem) => VectorElement(edge.source, edge.weight * elem.value))
-//  }
-//
-//  def rescale(v3: DataSetOLD[VectorElement], c: Double, r: Double) = {
-//    v3.map(elem => { VectorElement(elem.index, c * elem.value + (1 - c) * r) })
-//  }
-//
-//  def powerMethod(SGT: DataSetOLD[Edge], TG: DataSetOLD[Edge], d: DataSetOLD[VectorElement], c: Double, r: Double)(v: DataSetOLD[VectorElement]) = {
-//
-//    val v1 = d.join(v).where(_.index).isEqualTo(_.index)
-//      .map((leftElem, rightElem) => VectorElement(leftElem.index, leftElem.value * rightElem.value))
-//
-//    val v2 = sgtTimes(SGT, v1)
-//    val v3 = tgTimes(TG, v2)
-//    val nextV = rescale(v3, c, r)
-//
-//    nextV
-//  }
-//
-//  def getScalaPlan(numSubTasks: Int, sourceIncidenceMatrixPath: String, targetIncidenceMatrixPath: String, m: Int,
-//    outputPath: String) = {
-//
-//    val c = .85
-//    val r = 1.0 / m
-//
-//    val SGT = DataSource(sourceIncidenceMatrixPath, CsvInputFormat[Edge]())
-//    val TG = DataSource(targetIncidenceMatrixPath, CsvInputFormat[Edge]())
-//
-//    val d1 = SGT.map(edge => VectorElement(edge.target, edge.weight))
-//      .groupBy(_.index)
-//      .reduce(sumElements)
-//
-//    val d2 = tgTimes(TG, d1)
-//
-//    val d = d2.map(elem => VectorElement(elem.index, 1 / elem.value))
-//
-//    val initialV1 = d.map(elem => VectorElement(elem.index, elem.value * m))
-//    val initialV2 = sgtTimes(SGT, initialV1)
-//    val initialV3 = tgTimes(TG, initialV2)
-//    val initialV = rescale(initialV3, c, r)
-//
-//    val v = initialV.iterate(5, powerMethod(SGT, TG, d, c, r))
-//
-//    val output = v.write(outputPath, CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(output), "LineRank")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
index 25347ca..5416bb4 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/graph/TransitiveClosureNaive.scala
@@ -109,4 +109,4 @@ object TransitiveClosureNaive {
 			env.fromCollection(edgeData)
 		}
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
deleted file mode 100644
index b9f2264..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/iterative/TerminationCriterion.scala
+++ /dev/null
@@ -1,78 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.iterative
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-///**
-// * Example of using the bulk iteration with termination criterion with the
-// * scala api.
-// */
-//class TerminationCriterion extends Program with ProgramDescription with Serializable {
-//  override def getDescription() = {
-//    "Parameters: <maxNumberIterations> <output>"
-//  }
-//
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1))
-//  }
-//
-//  def getScalaPlan(maxNumberIterations: Int, resultOutput: String) = {
-//    val dataSource = CollectionDataSource[Double](List(1.0))
-//
-//    val halve = (partialSolution: DataSetOLD[Double]) => {
-//      partialSolution map { x => x /2 }
-//    }
-//
-//    val terminationCriterion = (prev: DataSetOLD[Double], cur: DataSetOLD[Double]) => {
-//      val diff = prev cross cur map { (valuePrev, valueCurrent) => math.abs(valuePrev - valueCurrent) }
-//      diff filter {
-//        difference => difference > 0.1
-//      }
-//    }
-//
-//    val iteration = dataSource.iterateWithTermination(maxNumberIterations, halve, terminationCriterion)
-//
-//
-//    val sink = iteration.write(resultOutput, CsvOutputFormat())
-//
-//    val plan = new ScalaPlan(Seq(sink))
-//    plan.setDefaultParallelism(1)
-//    plan
-//  }
-//}
-//
-//object RunTerminationCriterion {
-//  def main(args: Array[String]) {
-//    val tc = new TerminationCriterion
-//
-//    if(args.size < 2) {
-//      println(tc.getDescription())
-//      return
-//    }
-//    val plan = tc.getScalaPlan(args(0).toInt, args(1))
-//    LocalExecutor.execute(plan)
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
index 3e66275..95dcb9a 100644
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
+++ b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/ml/LinearRegression.scala
@@ -28,58 +28,56 @@ import org.apache.flink.examples.java.ml.util.LinearRegressionData
 import scala.collection.JavaConverters._
 
 /**
- * This example implements a basic Linear Regression  to solve the y = theta0 + theta1*x problem using batch gradient descent algorithm.
+ * This example implements a basic Linear Regression  to solve the y = theta0 + theta1*x problem
+ * using batch gradient descent algorithm.
  *
- * <p>
- * Linear Regression with BGD(batch gradient descent) algorithm is an iterative clustering algorithm and works as follows:<br>
- * Giving a data set and target set, the BGD try to find out the best parameters for the data set to fit the target set.
- * In each iteration, the algorithm computes the gradient of the cost function and use it to update all the parameters.
- * The algorithm terminates after a fixed number of iterations (as in this implementation)
+ * Linear Regression with BGD(batch gradient descent) algorithm is an iterative clustering
+ * algorithm and works as follows:
+ *
+ * Giving a data set and target set, the BGD try to find out the best parameters for the data set
+ * to fit the target set.
+ * In each iteration, the algorithm computes the gradient of the cost function and use it to
+ * update all the parameters.
+ * The algorithm terminates after a fixed number of iterations (as in this implementation).
  * With enough iteration, the algorithm can minimize the cost function and find the best parameters
- * This is the Wikipedia entry for the <a href = "http://en.wikipedia.org/wiki/Linear_regression">Linear regression</a> and <a href = "http://en.wikipedia.org/wiki/Gradient_descent">Gradient descent algorithm</a>.
- * 
- * <p>
- * This implementation works on one-dimensional data. And find the two-dimensional theta.<br>
- * It find the best Theta parameter to fit the target.
- * 
- * <p>
+ * This is the Wikipedia entry for the
+ * [[http://en.wikipedia.org/wiki/Linear_regression Linear regression]] and
+ * [[http://en.wikipedia.org/wiki/Gradient_descent Gradient descent algorithm]].
+ *
+ * This implementation works on one-dimensional data and finds the best two-dimensional theta to
+ * fit the target.
+ *
  * Input files are plain text files and must be formatted as follows:
- * <ul>
- * <li>Data points are represented as two double values separated by a blank character. The first one represent the X(the training data) and the second represent the Y(target).
- * Data points are separated by newline characters.<br>
- * For example <code>"-0.02 -0.04\n5.3 10.6\n"</code> gives two data points (x=-0.02, y=-0.04) and (x=5.3, y=10.6).
- * </ul>
- * 
- * <p>
+ *
+ *  - Data points are represented as two double values separated by a blank character. The first
+ *    one represent the X(the training data) and the second represent the Y(target). Data points are
+ *    separated by newline characters.
+ *    For example `"-0.02 -0.04\n5.3 10.6\n"`gives two data points
+ *    (x=-0.02, y=-0.04) and (x=5.3, y=10.6).
+ *
  * This example shows how to use:
- * <ul>
- * <li> Bulk iterations
- * <li> Broadcast variables in bulk iterations
- * <li> Custom Java objects (PoJos)
- * </ul>
+ *
+ *  - Bulk iterations
+ *  - Broadcast variables in bulk iterations
  */
 object LinearRegression {
 
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
 	def main(args: Array[String]) {
 		if (!parseParameters(args)) {
 			return
 		}
 
 		val env = ExecutionEnvironment.getExecutionEnvironment
-		val data: DataSet[Data] = getDataSet(env)
-		val parameters: DataSet[Params] = getParamsDataSet(env)
+		val data = getDataSet(env)
+		val parameters = getParamsDataSet(env)
+
 		val result = parameters.iterate(numIterations) { currentParameters =>
 			val newParameters = data
 				.map(new SubUpdate).withBroadcastSet(currentParameters, "parameters")
-				.reduce { (val1, val2) =>
-				val new_theta0: Double = val1._1.getTheta0 + val2._1.getTheta0
-				val new_theta1: Double = val1._1.getTheta1 + val2._1.getTheta1
-				val result: Params = new Params(new_theta0, new_theta1)
-				(result, val1._2 + val2._2)
-			}
+				.reduce { (p1, p2) =>
+          val result = p1._1 + p2._1
+				  (result, p1._2 + p2._2)
+			  }
 				.map { x => x._1.div(x._2) }
 			newParameters
 		}
@@ -88,73 +86,28 @@ object LinearRegression {
 			result.writeAsText(outputPath)
 		}
 		else {
-			result.print
+			result.print()
 		}
 		env.execute("Scala Linear Regression example")
 	}
 
-	// *************************************************************************
-	//     DATA TYPES
-	// *************************************************************************
 	/**
 	 * A simple data sample, x means the input, and y means the target.
 	 */
-	class Data extends Serializable {
-
-		def this(x: Double, y: Double) {
-			this()
-			this.x = x
-			this.y = y
-		}
-
-		override def toString: String = {
-			"(" + x + "|" + y + ")"
-		}
-
-		var x: Double = .0
-		var y: Double = .0
-	}
+  case class Data(var x: Double, var y: Double)
 
 	/**
 	 * A set of parameters -- theta0, theta1.
 	 */
-	class Params extends Serializable {
+  case class Params(theta0: Double, theta1: Double) {
+    def div(a: Int): Params = {
+      Params(theta0 / a, theta1 / a)
+    }
 
-		def this(x0: Double, x1: Double) {
-			this()
-			this.theta0 = x0
-			this.theta1 = x1
-		}
-
-		override def toString: String = {
-			theta0 + " " + theta1
-		}
-
-		def getTheta0: Double = {
-			theta0
-		}
-
-		def getTheta1: Double = {
-			theta1
-		}
-
-		def setTheta0(theta0: Double) {
-			this.theta0 = theta0
-		}
-
-		def setTheta1(theta1: Double) {
-			this.theta1 = theta1
-		}
-
-		def div(a: Integer): Params = {
-			this.theta0 = theta0 / a
-			this.theta1 = theta1 / a
-			return this
-		}
-
-		private var theta0: Double = .0
-		private var theta1: Double = .0
-	}
+    def +(other: Params) = {
+      Params(theta0 + other.theta0, theta1 + other.theta1)
+    }
+  }
 
 	// *************************************************************************
 	//     USER FUNCTIONS
@@ -163,24 +116,22 @@ object LinearRegression {
 	/**
 	 * Compute a single BGD type update for every parameters.
 	 */
-	class SubUpdate extends RichMapFunction[Data, Tuple2[Params, Integer]] {
+	class SubUpdate extends RichMapFunction[Data, (Params, Int)] {
 
-		private var parameters: Traversable[Params] = null
-		var parameter: Params = null
-		private var count: Int = 1
+		private var parameter: Params = null
 
 		/** Reads the parameters from a broadcast variable into a collection. */
 		override def open(parameters: Configuration) {
-			this.parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala
+			val parameters = getRuntimeContext.getBroadcastVariable[Params]("parameters").asScala
+      parameter = parameters.head
 		}
 
-		def map(in: Data): Tuple2[Params, Integer] = {
-			for (p <- parameters) {
-				this.parameter = p
-			}
-			val theta_0: Double = parameter.getTheta0 - 0.01 * ((parameter.getTheta0 + (parameter.getTheta1 * in.x)) - in.y)
-			val theta_1: Double = parameter.getTheta1 - 0.01 * (((parameter.getTheta0 + (parameter.getTheta1 * in.x)) - in.y) * in.x)
-			new Tuple2[Params, Integer](new Params(theta_0, theta_1), count)
+		def map(in: Data): (Params, Int) = {
+			val theta0 =
+        parameter.theta0 - 0.01 * ((parameter.theta0 + (parameter.theta1 * in.x)) - in.y)
+			val theta1 =
+        parameter.theta1 - 0.01 * (((parameter.theta0 + (parameter.theta1 * in.x)) - in.y) * in.x)
+			(Params(theta0, theta1), 1)
 		}
 	}
 
@@ -198,7 +149,7 @@ object LinearRegression {
 			if (programArguments.length == 3) {
 				dataPath = programArguments(0)
 				outputPath = programArguments(1)
-				numIterations = Integer.parseInt(programArguments(2))
+				numIterations = programArguments(2).toInt
 			}
 			else {
 				System.err.println("Usage: LinearRegression <data path> <result path> <num iterations>")
@@ -206,11 +157,13 @@ object LinearRegression {
 			}
 		}
 		else {
-			System.out.println("Executing Linear Regression example with default parameters and built-in default data.")
-			System.out.println("  Provide parameters to read input data from files.")
-			System.out.println("  See the documentation for the correct format of input files.")
-			System.out.println("  We provide a data generator to create synthetic input files for this program.")
-			System.out.println("  Usage: LinearRegression <data path> <result path> <num iterations>")
+      System.out.println("Executing Linear Regression example with default parameters and " +
+        "built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("  We provide a data generator to create synthetic input files for this " +
+        "program.")
+      System.out.println("  Usage: LinearRegression <data path> <result path> <num iterations>")
 		}
 		true
 	}
@@ -225,7 +178,7 @@ object LinearRegression {
 		}
 		else {
 			val data = LinearRegressionData.DATA map {
-				case Array(x, y) => new Data(x.asInstanceOf[Double], y.asInstanceOf[Double])
+				case Array(x, y) => Data(x.asInstanceOf[Double], y.asInstanceOf[Double])
 			}
 			env.fromCollection(data)
 		}
@@ -233,7 +186,7 @@ object LinearRegression {
 
 	private def getParamsDataSet(env: ExecutionEnvironment): DataSet[Params] = {
 		val params = LinearRegressionData.PARAMS map {
-			case Array(x, y) => new Params(x.asInstanceOf[Double], y.asInstanceOf[Double])
+			case Array(x, y) => Params(x.asInstanceOf[Double], y.asInstanceOf[Double])
 		}
 		env.fromCollection(params)
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
deleted file mode 100644
index 5e7d7f3..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/relational/RelationalQuery.scala
+++ /dev/null
@@ -1,107 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.relational;
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//
-///**
-// * The TPC-H is a decision support benchmark on relational data.
-// * Its documentation and the data generator (DBGEN) can be found
-// * on http://www.tpc.org/tpch/ .
-// *
-// * This Flink program implements a modified version of the query 3 of
-// * the TPC-H benchmark including one join, some filtering and an
-// * aggregation. The query resembles the following SQL statement:
-// * <pre>
-// * SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
-// *   FROM orders, lineitem
-// *   WHERE l_orderkey = o_orderkey
-// *     AND o_orderstatus = "X"
-// *     AND YEAR(o_orderdate) > Y
-// *     AND o_orderpriority LIKE "Z%"
-// *   GROUP BY l_orderkey, o_shippriority;
-// * </pre>
-// */
-//class RelationalQuery extends Program with ProgramDescription with Serializable {
-//
-//  case class Order(orderId: Int, status: Char, year: Int, orderPriority: String, shipPriority: Int)
-//  case class LineItem(orderId: Int, extendedPrice: Double)
-//  case class PrioritizedOrder(orderId: Int, shipPriority: Int, revenue: Double)
-//
-//
-//  def getScalaPlan(numSubTasks: Int, ordersInput: String, lineItemsInput: String, ordersOutput: String, status: Char = 'F', minYear: Int = 1993, priority: String = "5") = {
-//
-//    // ORDER intput: parse as CSV and select relevant fields
-//    val orders = DataSource(ordersInput, CsvInputFormat[(Int, String, String, String, String, String, String, Int)]("\n", '|'))
-//                         .map { t => Order(t._1, t._3.charAt(0), t._5.substring(0,4).toInt, t._6, t._8) }
-//
-//    // ORDER intput: parse as CSV and select relevant fields
-//    val lineItems = DataSource(lineItemsInput, CsvInputFormat[(Int, String, String, String, String, Double)]("\n", '|'))
-//                         .map { t => LineItem(t._1, t._6) }
-//
-//    // filter the orders input
-//    val filteredOrders = orders filter { o => o.status == status && o.year > minYear && o.orderPriority.startsWith(priority) }
-//
-//    // join the filteres result with the lineitem input
-//    val prioritizedItems = filteredOrders join lineItems where { _.orderId } isEqualTo { _.orderId } map { (o, li) => PrioritizedOrder(o.orderId, o.shipPriority, li.extendedPrice) }
-//
-//    // group by and sum the joined data
-//    val prioritizedOrders = prioritizedItems groupBy { pi => (pi.orderId, pi.shipPriority) } reduce { (po1, po2) => po1.copy(revenue = po1.revenue + po2.revenue) }
-//
-//    // write the result as csv
-//    val output = prioritizedOrders.write(ordersOutput, CsvOutputFormat("\n", "|"))
-//
-//    val plan = new ScalaPlan(Seq(output), "Relational Query")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//
-//  override def getDescription() = {
-//    "Parameters: <orders>, <lineitem>, <output>, <degree-of-parallelism>"
-//  }
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(3).toInt, args(0), args(1), args(2))
-//  }
-//}
-//
-//
-///**
-// * Entry point to make the example standalone runnable with the local executor
-// */
-//object RunRelationalQuery {
-//
-//  def main(args: Array[String]) {
-//    val query = new RelationalQuery
-//
-//    if (args.size < 4) {
-//      println(query.getDescription)
-//      return
-//    }
-//    val plan = query.getScalaPlan(args(3).toInt, args(0), args(1), args(2))
-//    LocalExecutor.execute(plan)
-//  }
-//}
-//

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
deleted file mode 100644
index 52bfc15..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/testing/KMeansForTest.scala
+++ /dev/null
@@ -1,105 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.testing
-//
-//// Uncomment if you need to rebuild it for PackagedProgramEndToEndTest
-////
-////import org.apache.flink.api.common.Program
-////import org.apache.flink.api.common.ProgramDescription
-////
-////import org.apache.flink.api.scala._
-////import org.apache.flink.api.scala.operators._
-////
-////
-////class KMeansForTest extends Program with ProgramDescription {
-////
-////  override def getPlan(args: String*) = {
-////    getScalaPlan(args(0).toInt, args(1), args(2), args(3), args(4).toInt)
-////  }
-////
-////  case class Point(x: Double, y: Double, z: Double) {
-////    def computeEuclidianDistance(other: Point) = other match {
-////      case Point(x2, y2, z2) => math.sqrt(math.pow(x - x2, 2) + math.pow(y - y2, 2) + math.pow(z - z2, 2))
-////    }
-////  }
-////
-////  case class Distance(dataPoint: Point, clusterId: Int, distance: Double)
-////
-////  def asPointSum = (pid: Int, dist: Distance) => dist.clusterId -> PointSum(1, dist.dataPoint)
-////
-////  //  def sumPointSums = (dataPoints: Iterator[(Int, PointSum)]) => dataPoints.reduce { (z, v) => z.copy(_2 = z._2 + v._2) }
-////  def sumPointSums = (dataPoints: Iterator[(Int, PointSum)]) => {
-////    dataPoints.reduce { (z, v) => z.copy(_2 = z._2 + v._2) }
-////  }
-////
-////
-////  case class PointSum(count: Int, pointSum: Point) {
-////    def +(that: PointSum) = that match {
-////      case PointSum(c, Point(x, y, z)) => PointSum(count + c, Point(x + pointSum.x, y + pointSum.y, z + pointSum.z))
-////    }
-////
-////    def toPoint() = Point(round(pointSum.x / count), round(pointSum.y / count), round(pointSum.z / count))
-////
-////    // Rounding ensures that we get the same results in a multi-iteration run
-////    // as we do in successive single-iteration runs, since the output format
-////    // only contains two decimal places.
-////    private def round(d: Double) = math.round(d * 100.0) / 100.0;
-////  }
-////
-////  def parseInput = (line: String) => {
-////    val PointInputPattern = """(\d+)\|-?(\d+\.\d+)\|-?(\d+\.\d+)\|-?(\d+\.\d+)\|""".r
-////    val PointInputPattern(id, x, y, z) = line
-////    (id.toInt, Point(x.toDouble, y.toDouble, z.toDouble))
-////  }
-////
-////  def formatOutput = (cid: Int, p: Point) => "%d|%.2f|%.2f|%.2f|".format(cid, p.x, p.y, p.z)
-////
-////  def computeDistance(p: (Int, Point), c: (Int, Point)) = {
-////    val ((pid, dataPoint), (cid, clusterPoint)) = (p, c)
-////    val distToCluster = dataPoint.computeEuclidianDistance(clusterPoint)
-////
-////    pid -> Distance(dataPoint, cid, distToCluster)
-////  }
-////
-////
-////  def getScalaPlan(numSubTasks: Int, dataPointInput: String, clusterInput: String, clusterOutput: String, numIterations: Int) = {
-////    val dataPoints = DataSource(dataPointInput, DelimitedInputFormat(parseInput))
-////    val clusterPoints = DataSource(clusterInput, DelimitedInputFormat(parseInput))
-////
-////    val finalCenters = clusterPoints.iterate(numIterations, { centers =>
-////
-////      val distances = dataPoints cross centers map computeDistance
-////      val nearestCenters = distances groupBy { case (pid, _) => pid } reduceGroup { ds => ds.minBy(_._2.distance) } map asPointSum.tupled
-////      val newCenters = nearestCenters groupBy { case (cid, _) => cid } reduceGroup sumPointSums map { case (cid, pSum) => cid -> pSum.toPoint() }
-////
-////      newCenters
-////    })
-////
-////    val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatOutput.tupled))
-////
-////    val plan = new ScalaPlan(Seq(output), "KMeans Iteration (ONLY FOR TESTING)")
-////    plan.setDefaultParallelism(numSubTasks)
-////    plan
-////  }
-////
-////  override def getDescription() = {
-////    "Parameters: [numSubStasksS] [dataPoints] [clusterCenters] [output] [numIterations]"
-////  }
-////}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
deleted file mode 100644
index f71b18d..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithCount.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.wordcount
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//
-///**
-// * Implementation of word count in Scala. This example uses the built in count function for tuples.
-// */
-//class WordCountWithCount extends WordCount {
-//
-//  override def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
-//    val input = TextFile(textInput)
-//
-//    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
-//    val counts = words groupBy { x => x } count()
-//
-//    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
-//
-//    val plan = new ScalaPlan(Seq(output), "Word Count")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//}
-//
-//
-///**
-// * Entry point to make the example standalone runnable with the local executor.
-// */
-//object RunWordCountWithCount {
-//  def main(args: Array[String]) {
-//    val wc = new WordCountWithCount
-//    if (args.size < 3) {
-//      println(wc.getDescription)
-//      return
-//    }
-//    val plan = wc.getScalaPlan(args(0).toInt, args(1), args(2))
-//    LocalExecutor.execute(plan)
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala b/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
deleted file mode 100644
index 2ee0c43..0000000
--- a/flink-examples/flink-scala-examples/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCountWithUserDefinedType.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.examples.scala.wordcount
-//
-//import org.apache.flink.client.LocalExecutor
-//import org.apache.flink.api.common.Program
-//import org.apache.flink.api.common.ProgramDescription
-//
-//import org.apache.flink.types.IntValue
-//import org.apache.flink.types.StringValue
-//
-//import org.apache.flink.api.scala._
-//import org.apache.flink.api.scala.operators._
-//
-//
-///**
-// * Implementation of word count in Scala, using a user defined type rather than one of the
-// * built-in supported types like primitives, tuples, or other (nested) case classes.
-// */
-//class WordCountWithUserDefinedType extends Program with Serializable {
-//
-//  def getScalaPlan(numSubTasks: Int, textInput: String, wordsOutput: String) = {
-//    val input = TextFile(textInput)
-//
-//    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } map { w => (new StringValue(w), new IntValue(1)) } }
-//
-//    val counts = words
-//      .groupBy { case (word, _) => word }
-//      .reduce { (w1, w2) => (w1._1, new IntValue(w1._2.getValue + w2._2.getValue)) }
-//
-//    val output = counts.write(wordsOutput, CsvOutputFormat("\n", " "))
-//
-//    val plan = new ScalaPlan(Seq(output), "Word Count (immutable)")
-//    plan.setDefaultParallelism(numSubTasks)
-//    plan
-//  }
-//
-//
-//  override def getPlan(args: String*) = {
-//    getScalaPlan(args(0).toInt, args(1), args(2))
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index 45db390..dfd9419 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -39,23 +39,23 @@ under the License.
 			<version>${project.version}</version>
 		</dependency>
 
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-core</artifactId>
-            <version>${project.version}</version>
-            <type>test-jar</type>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-java</artifactId>
-            <version>${project.version}</version>
-            <type>test-jar</type>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-java</artifactId>
 			<version>${project.version}</version>
@@ -108,7 +108,7 @@ under the License.
 				<version>3.1.4</version>
 				<executions>
 					<!-- Run scala compiler in the process-resources phase, so that dependencies on
-					    scala classes can be resolved later in the (Java) compile phase -->
+						scala classes can be resolved later in the (Java) compile phase -->
 					<execution>
 						<id>scala-compile-first</id>
 						<phase>process-resources</phase>
@@ -118,7 +118,7 @@ under the License.
 					</execution>
  
 					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
-					     scala classes can be resolved later in the (Java) test-compile phase -->
+						 scala classes can be resolved later in the (Java) test-compile phase -->
 					<execution>
 						<id>scala-test-compile</id>
 						<phase>process-test-resources</phase>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index d04f968..e973093 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -635,14 +635,12 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    *   val left: DataSet[(String, Int, Int)] = ...
    *   val right: DataSet[(Int, String, Int)] = ...
    *   val joined = left.join(right).where(0).isEqualTo(1) { (l, r) =>
-   *     if (l._2 > 4) {
-   *       Some((l._2, r._3))
-   *     } else {
-   *       None
-   *     }
+   *     (l._1, r._2)
    *   }
    * }}}
-   * This can be used to implement a filter directly in the join or to output more than one values:
+   * A join function with a [[Collector]] can be used to implement a filter directly in the join
+   * or to output more than one values. This type of join function does not return a value, instead
+   * values are emitted using the collector:
    * {{{
    *   val left: DataSet[(String, Int, Int)] = ...
    *   val right: DataSet[(Int, String, Int)] = ...
@@ -696,11 +694,12 @@ class DataSet[T: ClassTag](private[flink] val set: JavaDataSet[T]) {
    *   val right: DataSet[(Int, String, Int)] = ...
    *   val coGrouped = left.coGroup(right).where(0).isEqualTo(1) { (l, r) =>
    *     // l and r are of type TraversableOnce
-   *     Some((l.min, r.max))
+   *     (l.min, r.max)
    *   }
    * }}}
-   * This can be used to implement a filter directly in the coGroup or to output more than one
-   * values:
+   * A coGroup function with a [[Collector]] can be used to implement a filter directly in the
+   * coGroup or to output more than one values. This type of coGroup function does not return a
+   * value, instead values are emitted using the collector
    * {{{
    *   val left: DataSet[(String, Int, Int)] = ...
    *   val right: DataSet[(Int, String, Int)] = ...

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
index a7ca821..802fd09 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
@@ -144,11 +144,9 @@ trait GroupedDataSet[T] {
 }
 
 /**
- * /**
  * Private implementation for [[GroupedDataSet]] to keep the implementation details, i.e. the
  * parameters of the constructor, hidden.
  */
- */
 private[flink] class GroupedDataSetImpl[T: ClassTag](
     private val set: JavaDataSet[T],
     private val keys: Keys[T])
@@ -256,7 +254,7 @@ private[flink] class GroupedDataSetImpl[T: ClassTag](
   }
 
   def reduceGroup[R: TypeInformation: ClassTag](
-                                                 fun: (TraversableOnce[T]) => R): DataSet[R] = {
+      fun: (TraversableOnce[T]) => R): DataSet[R] = {
     Validate.notNull(fun, "Group reduce function must not be null.")
     val reducer = new GroupReduceFunction[T, R] {
       def reduce(in: java.lang.Iterable[T], out: Collector[R]) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
deleted file mode 100644
index 2426bc9..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ComputeEdgeDegreesITCase.java
+++ /dev/null
@@ -1,39 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.configuration.Configuration;
-//import org.apache.flink.examples.scala.graph.ComputeEdgeDegrees;
-//
-//public class ComputeEdgeDegreesITCase extends org.apache.flink.test.recordJobTests.ComputeEdgeDegreesITCase {
-//
-//	public ComputeEdgeDegreesITCase(Configuration config) {
-//		super(config);
-//	}
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		ComputeEdgeDegrees computeDegrees = new ComputeEdgeDegrees();
-//		return computeDegrees.getScalaPlan(
-//				config.getInteger("ComputeEdgeDegreesTest#NumSubtasks", DOP),
-//				edgesPath, resultPath);
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
index 69a5c9a..71a7e23 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
@@ -1,34 +1,60 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.examples.scala.graph.ConnectedComponents;
-//
-//public class ConnectedComponentsITCase extends org.apache.flink.test.iterative.ConnectedComponentsITCase {
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		ConnectedComponents cc = new ConnectedComponents();
-//		Plan plan = cc.getScalaPlan(verticesPath, edgesPath, resultPath, 100);
-//		plan.setDefaultParallelism(DOP);
-//		return plan;
-//	}
-//}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.test.exampleScalaPrograms;
+
+import org.apache.flink.examples.scala.graph.ConnectedComponents;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.BufferedReader;
+
+public class ConnectedComponentsITCase extends JavaProgramTestBase {
+	
+	private static final long SEED = 0xBADC0FFEEBEEFL;
+	
+	private static final int NUM_VERTICES = 1000;
+	
+	private static final int NUM_EDGES = 10000;
+
+	
+	private String verticesPath;
+	private String edgesPath;
+	private String resultPath;
+	
+	
+	@Override
+	protected void preSubmit() throws Exception {
+		verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES));
+		edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
+		resultPath = getTempFilePath("results");
+	}
+	
+	@Override
+	protected void testProgram() throws Exception {
+		ConnectedComponents.main(new String[] {verticesPath, edgesPath, resultPath, "100"});
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		for (BufferedReader reader : getResultReader(resultPath)) {
+			ConnectedComponentsData.checkOddEvenResult(reader);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleBasicITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleBasicITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleBasicITCase.java
new file mode 100644
index 0000000..5c19876
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleBasicITCase.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.exampleScalaPrograms;
+
+import org.apache.flink.examples.scala.graph.EnumTrianglesBasic;
+import org.apache.flink.test.testdata.EnumTriangleData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class EnumTriangleBasicITCase extends JavaProgramTestBase {
+	
+	protected String edgePath;
+	protected String resultPath;
+	
+	@Override
+	protected void preSubmit() throws Exception {
+		edgePath = createTempFile("edges", EnumTriangleData.EDGES);
+		resultPath = getTempDirPath("triangles");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_ID, resultPath);
+	}
+	
+	@Override
+	protected void testProgram() throws Exception {
+		EnumTrianglesBasic.main(new String[] { edgePath, resultPath });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleOptITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleOptITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleOptITCase.java
new file mode 100644
index 0000000..944aaf4
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleOptITCase.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.exampleScalaPrograms;
+
+import org.apache.flink.examples.scala.graph.EnumTrianglesOpt;
+import org.apache.flink.test.testdata.EnumTriangleData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class EnumTriangleOptITCase extends JavaProgramTestBase {
+	
+	protected String edgePath;
+	protected String resultPath;
+	
+	@Override
+	protected void preSubmit() throws Exception {
+		edgePath = createTempFile("edges", EnumTriangleData.EDGES);
+		resultPath = getTempDirPath("triangles");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_DEGREE, resultPath);
+	}
+	
+	@Override
+	protected void testProgram() throws Exception {
+		EnumTrianglesOpt.main(new String[] { edgePath, resultPath });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
deleted file mode 100644
index ab8c563..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTrianglesOnEdgesWithDegreesITCase.java
+++ /dev/null
@@ -1,39 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.configuration.Configuration;
-//import org.apache.flink.examples.scala.graph.EnumTrianglesOnEdgesWithDegrees;
-//
-//public class EnumTrianglesOnEdgesWithDegreesITCase extends org.apache.flink.test.recordJobTests.EnumTrianglesOnEdgesWithDegreesITCase {
-//
-//	public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) {
-//		super(config);
-//	}
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		EnumTrianglesOnEdgesWithDegrees enumTriangles = new EnumTrianglesOnEdgesWithDegrees();
-//		return enumTriangles.getScalaPlan(
-//				config.getInteger("EnumTrianglesTest#NumSubtasks", DOP),
-//				edgesPath, resultPath);
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
new file mode 100644
index 0000000..2369a4b
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.exampleScalaPrograms;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.examples.scala.graph.PageRankBasic;
+import org.apache.flink.test.testdata.PageRankData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+
+@RunWith(Parameterized.class)
+public class PageRankITCase extends JavaProgramTestBase {
+	
+	private static int NUM_PROGRAMS = 2;
+	
+	private int curProgId = config.getInteger("ProgramId", -1);
+	
+	private String verticesPath;
+	private String edgesPath;
+	private String resultPath;
+	private String expectedResult;
+	
+	public PageRankITCase(Configuration config) {
+		super(config);
+	}
+	
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+		verticesPath = createTempFile("vertices.txt", PageRankData.VERTICES);
+		edgesPath = createTempFile("edges.txt", PageRankData.EDGES);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		expectedResult = runProgram(curProgId);
+	}
+	
+	@Override
+	protected void postSubmit() throws Exception {
+		compareKeyValueParisWithDelta(expectedResult, resultPath, " ", 0.01);
+	}
+	
+	@Parameters
+	public static Collection<Object[]> getConfigurations() throws FileNotFoundException, IOException {
+
+		LinkedList<Configuration> tConfigs = new LinkedList<Configuration>();
+
+		for(int i=1; i <= NUM_PROGRAMS; i++) {
+			Configuration config = new Configuration();
+			config.setInteger("ProgramId", i);
+			tConfigs.add(config);
+		}
+		
+		return toParameterList(tConfigs);
+	}
+	
+
+	public String runProgram(int progId) throws Exception {
+		
+		switch(progId) {
+		case 1: {
+			PageRankBasic.main(new String[] {verticesPath, edgesPath, resultPath, PageRankData.NUM_VERTICES+"", "3"});
+			return PageRankData.RANKS_AFTER_3_ITERATIONS;
+		}
+		case 2: {
+			// start with a very high number of iteration such that the dynamic convergence criterion must handle termination
+			PageRankBasic.main(new String[] {verticesPath, edgesPath, resultPath, PageRankData.NUM_VERTICES+"", "1000"});
+			return PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE;
+		}
+		
+		default: 
+			throw new IllegalArgumentException("Invalid program id");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
deleted file mode 100644
index ca5a707..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/RelationalQueryITCase.java
+++ /dev/null
@@ -1,49 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.configuration.Configuration;
-//import org.apache.flink.examples.scala.relational.RelationalQuery;
-//import org.junit.runner.RunWith;
-//import org.junit.runners.Parameterized;
-//
-//import java.util.Locale;
-//
-//@RunWith(Parameterized.class)
-//public class RelationalQueryITCase extends org.apache.flink.test.recordJobTests.TPCHQuery3ITCase {
-//
-//	public RelationalQueryITCase(Configuration config) {
-//		super(config);
-//		Locale.setDefault(Locale.US);
-//	}
-//
-//	@Override
-//	protected Plan getTestJob()  {
-//
-//		RelationalQuery tpch3 = new RelationalQuery();
-//		return tpch3.getScalaPlan(
-//				config.getInteger("dop", 1),
-//				ordersPath,
-//				lineitemsPath,
-//				resultPath,
-//				'F', 1993, "5");
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
deleted file mode 100644
index 4d0eb24..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureNaiveITCase.java
+++ /dev/null
@@ -1,54 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.examples.scala.graph.TransitiveClosureNaive;
-//import org.apache.flink.test.util.RecordAPITestBase;
-//
-//public class TransitiveClosureNaiveITCase extends RecordAPITestBase {
-//
-//	protected String verticesPath = null;
-//	protected String edgesPath = null;
-//	protected String resultPath = null;
-//
-//	private static final String VERTICES = "0\n1\n2";
-//	private static final String EDGES = "0|1\n1|2";
-//	private static final String EXPECTED = "0|0|0\n0|1|1\n0|2|2\n1|1|0\n1|2|1\n2|2|0";
-//
-//	@Override
-//	protected void preSubmit() throws Exception {
-//		verticesPath = createTempFile("vertices.txt", VERTICES);
-//		edgesPath = createTempFile("edges.txt", EDGES);
-//		resultPath = getTempDirPath("transitiveClosure");
-//	}
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		TransitiveClosureNaive transitiveClosureNaive = new TransitiveClosureNaive();
-//		// "2" is the number of iterations here
-//		return transitiveClosureNaive.getScalaPlan(DOP, 2, verticesPath, edgesPath, resultPath);
-//	}
-//
-//	@Override
-//	protected void postSubmit() throws Exception {
-//		compareResultsByLinesInMemory(EXPECTED, resultPath);
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
index 63c598c..2d8ad31 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
@@ -1,32 +1,49 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.examples.scala.relational.WebLogAnalysis;
-//
-//public class WebLogAnalysisITCase extends org.apache.flink.test.recordJobTests.WebLogAnalysisITCase {
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		WebLogAnalysis webLogAnalysis = new WebLogAnalysis();
-//		return webLogAnalysis.getScalaPlan(DOP, docsPath, ranksPath, visitsPath, resultPath);
-//	}
-//}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.exampleScalaPrograms;
+
+
+import org.apache.flink.examples.scala.relational.WebLogAnalysis;
+import org.apache.flink.test.testdata.WebLogAnalysisData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class WebLogAnalysisITCase extends JavaProgramTestBase {
+
+	private String docsPath;
+	private String ranksPath;
+	private String visitsPath;
+	private String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		docsPath = createTempFile("docs", WebLogAnalysisData.DOCS);
+		ranksPath = createTempFile("ranks", WebLogAnalysisData.RANKS);
+		visitsPath = createTempFile("visits", WebLogAnalysisData.VISITS);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WebLogAnalysisData.EXCEPTED_RESULT, resultPath);
+	}
+	@Override
+	protected void testProgram() throws Exception {
+		WebLogAnalysis.main(new String[]{docsPath, ranksPath, visitsPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
deleted file mode 100644
index 94ec224..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountPactValueITCase.java
+++ /dev/null
@@ -1,33 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.examples.scala.wordcount.WordCountWithUserDefinedType;
-//
-//
-//public class WordCountPactValueITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		WordCountWithUserDefinedType wc = new WordCountWithUserDefinedType();
-//		return wc.getScalaPlan(DOP, textPath, resultPath);
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/31ed0c4c/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
deleted file mode 100644
index 5f53f72..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountWithCountFunctionITCase.java
+++ /dev/null
@@ -1,31 +0,0 @@
-///**
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//
-//package org.apache.flink.test.exampleScalaPrograms;
-//
-//import org.apache.flink.api.common.Plan;
-//import org.apache.flink.examples.scala.wordcount.WordCountWithCount;
-//
-//public class WordCountWithCountFunctionITCase extends org.apache.flink.test.recordJobTests.WordCountITCase {
-//
-//	@Override
-//	protected Plan getTestJob() {
-//		return new WordCountWithCount().getScalaPlan(DOP, textPath, resultPath);
-//	}
-//}