You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@wayang.apache.org by be...@apache.org on 2021/09/22 15:21:41 UTC

[incubator-wayang] branch WAYANG-28 created (now 6cfa2d8)

This is an automated email from the ASF dual-hosted git repository.

bertty pushed a change to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git.


      at 6cfa2d8  Merge pull request #29 from apache/debugger-sidecar

This branch includes the following new commits:

     new 12d9afa  [WAYANG-28] Add javadoc to tags package
     new 0890650  [WAYANG-28] Add javadoc to tagger package
     new e8d1d89  [WAYANG-28] template of javadoc to sniffer package
     new 428d328  [WAYANG-28] javadoc to sniffer package
     new dd11607  [WAYANG-28] creation of module wayang-hackit-api
     new 5a4815c  [WAYANG-31] split DataQuantaBuilder class on several class
     new d39ab4f  [WAYANG-31] structure the code on different way
     new 26af462  [WAYANG-31] DataQuanta was refactored as abstract class
     new 99a6665  [WAYANG-31] small correction to be extensible
     new 4cdd676  [WAYANG-28] creation a base of DataQuantaHackit and extension of test
     new 6c51e71  [WAYANG-28] change the list for set when it handle the collections of tags
     new ad7f451  [WAYANG-28] update the structure of class in the Tagger function wrappers
     new 5e26844  [WAYANG-28] use base java function on the tagger template
     new a272d73  [WAYANG-28] seed version of the Hackit API
     new 6cfa2d8  Merge pull request #29 from apache/debugger-sidecar

The 15 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


[incubator-wayang] 11/15: [WAYANG-28] change the list for set when it handle the collections of tags

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 6c51e71cc8a7e4fa858fd9908be676fceeaef374
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Mon May 17 12:19:12 2021 -0400

    [WAYANG-28] change the list for set when it handle the collections of tags
---
 .../plugin/hackit/core/tagger/HackitTagger.java    | 16 +++++++--------
 .../plugin/hackit/core/tuple/HackitTuple.java      | 10 ++++++++++
 .../plugin/hackit/core/tuple/header/Header.java    | 23 ++++++++++++++++++++--
 3 files changed, 39 insertions(+), 10 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
index bb171ef..e172903 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
@@ -24,8 +24,10 @@ import org.apache.wayang.plugin.hackit.core.tuple.header.Header;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 /**
  * HackitTagger is class where is allocated all the logic that need to be perform during the
@@ -39,15 +41,13 @@ public class HackitTagger implements Serializable {
      * {@link List} of {@link HackitTag} that are added previous of the execution of the
      * original function
      */
-    //TODO: It may change by a set
-    protected List<HackitTag> pre_tags;
+    protected Set<HackitTag> pre_tags;
 
     /**
      * {@link List} of {@link HackitTag} that are added after of the execution of the
      * original function
      */
-    //TODO: It may change by a set
-    protected List<HackitTag> post_tags;
+    protected Set<HackitTag> post_tags;
 
     /**
      * Default Construct
@@ -62,7 +62,7 @@ public class HackitTagger implements Serializable {
      */
     public HackitTagger addPreTag(HackitTag tag){
         if(this.pre_tags == null){
-            this.pre_tags = new ArrayList<>();
+            this.pre_tags = new HashSet<>();
         }
         this.pre_tags.add(tag);
         return this;
@@ -76,7 +76,7 @@ public class HackitTagger implements Serializable {
      */
     public HackitTagger addPostTag(HackitTag tag){
         if(this.post_tags == null){
-            this.post_tags = new ArrayList<>();
+            this.post_tags = new HashSet<>();
         }
         this.post_tags.add(tag);
         return this;
@@ -108,9 +108,9 @@ public class HackitTagger implements Serializable {
      * @param tuple is {@link HackitTuple} where the tags will be added
      * @param tags {@link List} of {@link HackitTag}'s that will add to {@link HackitTuple}
      */
-    public void taggingTuple(HackitTuple tuple, List<HackitTag> tags){
+    public void taggingTuple(HackitTuple tuple, Set<HackitTag> tags){
         //TODO: change this code for an efficient one
-        tags.stream().forEach(tag -> tuple.addTag(tag.getInstance()));
+        tuple.addTag(tags);
     }
 
     /**
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/HackitTuple.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/HackitTuple.java
index 8c2cc51..7e36fa0 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/HackitTuple.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/HackitTuple.java
@@ -24,6 +24,7 @@ import org.apache.wayang.plugin.hackit.core.tuple.header.HeaderBuilder;
 
 import java.io.Serializable;
 import java.util.Iterator;
+import java.util.Set;
 
 /**
  * HackitTuple is the base of Hackit because is the basic structure where the needed elements are added
@@ -123,6 +124,15 @@ public class HackitTuple<K, T> implements Serializable, ActionGroup {
     }
 
     /**
+     * add a {@link Set} of {@lin HackitTag} on the {@link Header}
+     *
+     * @param tags {@link HackitTag} that it will need at some point on the process
+     */
+    public void addTag(Set<HackitTag> tags){
+        this.header.addTag(tags);
+    }
+
+    /**
      * get a {@link Iterator} of the currents {@link HackitTag} that are inside of the {@link Header}
      * @return {@link Iterator} of tags
      */
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/header/Header.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/header/Header.java
index 2d3577b..c08a2f0 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/header/Header.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tuple/header/Header.java
@@ -23,8 +23,10 @@ import org.apache.wayang.plugin.hackit.core.tags.HackitTag;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Header is the container of the metadata asociated to one {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
@@ -49,7 +51,7 @@ public abstract class Header<K> implements Serializable, ActionGroup {
      * tags added to the header, this describe some action that need to be apply to the
      * {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
      */
-    private List<HackitTag> tags;
+    private Set<HackitTag> tags;
 
     /**
      * during the process of adding news {@link HackitTag} could add a new {@link org.apache.wayang.plugin.hackit.core.action.Action} at the header, and this
@@ -121,7 +123,7 @@ public abstract class Header<K> implements Serializable, ActionGroup {
     public void addTag(HackitTag tag){
         //TODO: could be better to use an Set because it just saving uniques elements
         if(this.tags == null){
-            this.tags = new ArrayList<>();
+            this.tags = new HashSet<>();
         }
         this.tags.add(tag);
         //update all the possible actions on the {@link ActionGroup}
@@ -130,6 +132,23 @@ public abstract class Header<K> implements Serializable, ActionGroup {
     }
 
     /**
+     * do exactly the same of {@link #addTag(HackitTag)} but adding all the element at the same time
+     *
+     * @param tags {@link Set} of {@link HackitTag} that will be added
+     */
+    public void addTag(Set<HackitTag> tags){
+        if(this.tags == null){
+            this.tags = new HashSet<>();
+        }
+        this.tags.addAll(tags);
+        tags.stream()
+            .forEach(
+                this::updateActionVector
+            )
+        ;
+    }
+
+    /**
      * remove all the tags from the header, and set all the possible options as false
      */
     public void clearTags(){

[incubator-wayang] 10/15: [WAYANG-28] creation a base of DataQuantaHackit and extension of test

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 4cdd676f519952aba107e80532e1bba220ea1ddf
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Fri May 14 08:55:46 2021 -0400

    [WAYANG-28] creation a base of DataQuantaHackit and extension of test
---
 .../wayang-hackit/wayang-hackit-api/pom.xml        |  30 +
 .../plugin/hackit/api/DataQuantaHackit.scala       | 395 ++++++++++++
 .../java/org/apache/wayang/api/JavaApiTest.java    | 711 ---------------------
 .../test/scala/org/apache/wayang/api/ApiTest.scala | 575 -----------------
 .../plugin/hackit/api/ApiExtensionTest.scala       |  41 ++
 .../wayang-hackit-api/src/test/wayang.properties   |  18 +
 6 files changed, 484 insertions(+), 1286 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
index 548f9e6..ae95199 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
@@ -30,6 +30,11 @@
 
     <artifactId>wayang-hackit-api</artifactId>
 
+    <properties>
+        <java-module-name>org.apache.wayang.plugin.hackit.api</java-module-name>
+        <spark.version>2.4.0</spark.version>
+    </properties>
+
     <dependencyManagement>
         <dependencies>
             <dependency>
@@ -50,6 +55,31 @@
         </dependency>
         <dependency>
             <groupId>org.apache.wayang</groupId>
+            <artifactId>wayang-api-scala-java_2.11</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.wayang</groupId>
+            <artifactId>wayang-spark_${scala.mayor.version}</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.mayor.version}</artifactId>
+            <version>${spark.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.wayang</groupId>
+            <artifactId>wayang-sqlite3</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.wayang</groupId>
             <artifactId>wayang-java</artifactId>
             <version>0.6.0-SNAPSHOT</version>
             <scope>test</scope>
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala
new file mode 100644
index 0000000..3f76c7b
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala
@@ -0,0 +1,395 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.plugin.hackit.api
+
+import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, groupedDataSetType, groupedDataUnitType}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaCreator, KeyedDataQuanta}
+import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
+import org.apache.wayang.basic.function.ProjectionDescriptor
+import org.apache.wayang.basic.operators.{CartesianOperator, CoGroupOperator, CountOperator, DistinctOperator, FilterOperator, FlatMapOperator, GlobalMaterializedGroupOperator, GlobalReduceOperator, IntersectOperator, JoinOperator, MapOperator, MapPartitionsOperator, MaterializedGroupByOperator, ReduceByOperator, SampleOperator, SortOperator, UnionAllOperator, ZipWithIdOperator}
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction, SerializablePredicate}
+import org.apache.wayang.core.function.{FlatMapDescriptor, MapPartitionsDescriptor, PredicateDescriptor, ReduceDescriptor, TransformationDescriptor}
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
+import org.apache.wayang.core.plan.wayangplan.{ElementaryOperator, Operator, OutputSlot, WayangPlan}
+
+import java.lang
+import java.lang.{Iterable => JavaIterable}
+import java.util.function.IntUnaryOperator
+import scala.reflect.ClassTag
+
+/**
+ * Represents an intermediate result/data flow edge in a [[WayangPlan]].
+ *
+ * @param operator    a unary [[Operator]] that produces this instance
+ * @param ev$1        the data type of the elements in this instance
+ * @param planBuilder keeps track of the [[WayangPlan]] being build
+ */
+class DataQuantaHackit[Out: ClassTag]
+                            (override val operator: ElementaryOperator, outputIndex: Int = 0)
+                            (implicit override val  planBuilder: PlanBuilder)
+                    extends DataQuanta[Out](operator, outputIndex) {
+
+  /**
+   * Feed this instance into a [[MapOperator]].
+   *
+   * @param udf     a Java 8 lambda expression as UDF for the [[MapOperator]]
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[MapOperator]]'s output
+   */
+  override def mapJava[NewOut: ClassTag](udf: SerializableFunction[Out, NewOut],
+                                         udfLoad: LoadProfileEstimator = null): DataQuantaHackit[NewOut] = {
+    val lala = new SerializableFunction[Out, NewOut] {
+      override def apply(t: Out): NewOut = {
+        println(t)
+        udf.apply(t)
+      }
+    }
+    val mapOperator = new MapOperator(new TransformationDescriptor(
+      lala, basicDataUnitType[Out], basicDataUnitType[NewOut], udfLoad
+    ))
+    this.connectTo(mapOperator, 0)
+    DataQuantaHackit.wrap[NewOut](mapOperator)
+  }
+
+  /**
+   * Feed this instance into a [[MapPartitionsOperator]].
+   *
+   * @param udf         a Java 8 lambda expression as UDF for the [[MapPartitionsOperator]]
+   * @param selectivity selectivity of the UDF
+   * @param udfLoad     optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[MapOperator]]'s output
+   */
+  override def mapPartitionsJava[NewOut: ClassTag](udf: SerializableFunction[JavaIterable[Out], JavaIterable[NewOut]],
+                                                   selectivity: ProbabilisticDoubleInterval = null,
+                                                   udfLoad: LoadProfileEstimator = null): DataQuantaHackit[NewOut] = {
+    val mapOperator = new MapPartitionsOperator(
+      new MapPartitionsDescriptor(udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad)
+    )
+    this.connectTo(mapOperator, 0)
+    DataQuantaHackit.wrap[NewOut](mapOperator)
+  }
+
+  /**
+   * Feed this instance into a [[MapOperator]] with a [[ProjectionDescriptor]].
+   *
+   * @param fieldNames names of the fields to be projected
+   * @return a new instance representing the [[MapOperator]]'s output
+   */
+  override def project[NewOut: ClassTag](fieldNames: Seq[String]): DataQuantaHackit[NewOut] = {
+    val projectionOperator = new MapOperator(
+      new ProjectionDescriptor(basicDataUnitType[Out], basicDataUnitType[NewOut], fieldNames: _*)
+    )
+    this.connectTo(projectionOperator, 0)
+    DataQuantaHackit.wrap[NewOut](projectionOperator)
+  }
+
+  /**
+   * Feed this instance into a [[FilterOperator]].
+   *
+   * @param udf         UDF for the [[FilterOperator]]
+   * @param sqlUdf      UDF as SQL `WHERE` clause
+   * @param selectivity selectivity of the UDF
+   * @param udfLoad     optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[FilterOperator]]'s output
+   */
+  override def filterJava(udf: SerializablePredicate[Out],
+                          sqlUdf: String = null,
+                          selectivity: ProbabilisticDoubleInterval = null,
+                          udfLoad: LoadProfileEstimator = null): DataQuantaHackit[Out] = {
+    val filterOperator = new FilterOperator(new PredicateDescriptor(
+      udf, this.output.getType.getDataUnitType.toBasicDataUnitType, selectivity, udfLoad
+    ).withSqlImplementation(sqlUdf))
+    this.connectTo(filterOperator, 0)
+    DataQuantaHackit.wrap[Out](filterOperator)
+  }
+
+  /**
+   * Feed this instance into a [[FlatMapOperator]].
+   *
+   * @param udf         a Java 8 lambda expression as UDF for the [[FlatMapOperator]]
+   * @param selectivity selectivity of the UDF
+   * @param udfLoad     optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[FlatMapOperator]]'s output
+   */
+  override def flatMapJava[NewOut: ClassTag](udf: SerializableFunction[Out, JavaIterable[NewOut]],
+                                             selectivity: ProbabilisticDoubleInterval = null,
+                                             udfLoad: LoadProfileEstimator = null): DataQuantaHackit[NewOut] = {
+    val flatMapOperator = new FlatMapOperator(new FlatMapDescriptor(
+      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad
+    ))
+    this.connectTo(flatMapOperator, 0)
+    DataQuantaHackit.wrap[NewOut](flatMapOperator)
+  }
+
+
+  /**
+   * Feed this instance into a [[SampleOperator]].
+   *
+   * @param sampleSizeFunction absolute size of the sample as a function of the current iteration number
+   * @param datasetSize        optional size of the dataset to be sampled
+   * @param sampleMethod       the [[SampleOperator.Methods]] to use for sampling
+   * @return a new instance representing the [[FlatMapOperator]]'s output
+   */
+  override def sampleDynamicJava(sampleSizeFunction: IntUnaryOperator,
+                                 datasetSize: Long = SampleOperator.UNKNOWN_DATASET_SIZE,
+                                 seed: Option[Long] = None,
+                                 sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuantaHackit[Out] = {
+    if (seed.isEmpty) {
+      val sampleOperator = new SampleOperator(
+        sampleSizeFunction,
+        dataSetType[Out],
+        sampleMethod
+      )
+      sampleOperator.setDatasetSize(datasetSize)
+      this.connectTo(sampleOperator, 0)
+      DataQuantaHackit.wrap[Out](sampleOperator)
+    }
+    else {
+      val sampleOperator = new SampleOperator(
+        sampleSizeFunction,
+        dataSetType[Out],
+        sampleMethod,
+        seed.get
+      )
+      sampleOperator.setDatasetSize(datasetSize)
+      this.connectTo(sampleOperator, 0)
+      DataQuantaHackit.wrap[Out](sampleOperator)
+    }
+  }
+
+  /**
+   * Assigns this instance a key extractor, which enables some key-based operations.
+   *
+   * @see KeyedDataQuanta
+   * @param keyExtractor extracts the key from the [[DataQuantaDefault]]
+   * @return the [[KeyedDataQuanta]]
+   */
+  //TODO validate this implementation
+  override def keyByJava[Key: ClassTag](keyExtractor: SerializableFunction[Out, Key]) : KeyedDataQuanta[Out, Key] = {
+    new KeyedDataQuanta[Out, Key](this, keyExtractor)
+  }
+
+  /**
+   * Feed this instance into a [[ReduceByOperator]].
+   *
+   * @param keyUdf  UDF to extract the grouping key from the data quanta
+   * @param udf     aggregation UDF for the [[ReduceByOperator]]
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[ReduceByOperator]]'s output
+   */
+  override def reduceByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
+                                              udf: SerializableBinaryOperator[Out],
+                                              udfLoad: LoadProfileEstimator = null)
+  : DataQuantaHackit[Out] = {
+    val reduceByOperator = new ReduceByOperator(
+      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
+    )
+    this.connectTo(reduceByOperator, 0)
+    DataQuantaHackit.wrap[Out](reduceByOperator)
+  }
+
+  /**
+   * Feed this instance into a [[MaterializedGroupByOperator]].
+   *
+   * @param keyUdf     UDF to extract the grouping key from the data quanta
+   * @param keyUdfLoad optional [[LoadProfileEstimator]] for the `keyUdf`
+   * @return a new instance representing the [[MaterializedGroupByOperator]]'s output
+   */
+  override def groupByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
+                                             keyUdfLoad: LoadProfileEstimator = null): DataQuantaHackit[java.lang.Iterable[Out]] = {
+    val groupByOperator = new MaterializedGroupByOperator(
+      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key], keyUdfLoad),
+      dataSetType[Out],
+      groupedDataSetType[Out]
+    )
+    this.connectTo(groupByOperator, 0)
+    DataQuantaHackit.wrap[java.lang.Iterable[Out]](groupByOperator)
+  }
+
+  /**
+   * Feed this instance into a [[GlobalReduceOperator]].
+   *
+   * @param udf     aggregation UDF for the [[GlobalReduceOperator]]
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[GlobalReduceOperator]]'s output
+   */
+  override def reduceJava(udf: SerializableBinaryOperator[Out],
+                          udfLoad: LoadProfileEstimator = null): DataQuantaHackit[Out] = {
+    val globalReduceOperator = new GlobalReduceOperator(
+      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
+    )
+    this.connectTo(globalReduceOperator, 0)
+    DataQuantaHackit.wrap[Out](globalReduceOperator)
+  }
+
+  /**
+   * Feeds this and a further instance into a [[JoinOperator]].
+   *
+   * @param thisKeyUdf UDF to extract keys from data quanta in this instance
+   * @param that       the other instance
+   * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
+   * @return a new instance representing the [[JoinOperator]]'s output
+   */
+  override def joinJava[ThatOut: ClassTag, Key: ClassTag](thisKeyUdf: SerializableFunction[Out, Key], that: DataQuanta[ThatOut], thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuantaHackit[WayangTuple2[Out, ThatOut]] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val joinOperator = new JoinOperator(
+      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
+    )
+    this.connectTo(joinOperator, 0)
+    that.connectTo(joinOperator, 1)
+    DataQuantaHackit.wrap[WayangTuple2[Out, ThatOut]](joinOperator)
+  }
+
+  /**
+   * Feeds this and a further instance into a [[CoGroupOperator]].
+   *
+   * @param thisKeyUdf UDF to extract keys from data quanta in this instance
+   * @param that       the other instance
+   * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
+   * @return a new instance representing the [[CoGroupOperator]]'s output
+   */
+  override def coGroupJava[ThatOut: ClassTag, Key: ClassTag](thisKeyUdf: SerializableFunction[Out, Key], that: DataQuanta[ThatOut], thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuantaHackit[WayangTuple2[JavaIterable[Out], JavaIterable[ThatOut]]] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val coGroupOperator = new CoGroupOperator(
+      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
+    )
+    this.connectTo(coGroupOperator, 0)
+    that.connectTo(coGroupOperator, 1)
+    DataQuantaHackit.wrap[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]](coGroupOperator)
+  }
+
+
+  /**
+   * Feeds this and a further instance into a [[SortOperator]].
+   *
+   * @param keyUdf UDF to extract key from data quanta in this instance
+   * @return a new instance representing the [[SortOperator]]'s output
+   */
+  override def sortJava[Key: ClassTag]
+  (keyUdf: SerializableFunction[Out, Key])
+  : DataQuantaHackit[Out] = {
+    val sortOperator = new SortOperator(new TransformationDescriptor(
+      keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]))
+    this.connectTo(sortOperator, 0)
+    DataQuantaHackit.wrap[Out](sortOperator)
+  }
+
+
+  /**
+   * Feed this instance into a [[GlobalMaterializedGroupOperator]].
+   *
+   * @return a new instance representing the [[GlobalMaterializedGroupOperator]]'s output
+   */
+  override def group(): DataQuantaHackit[JavaIterable[Out]] = {
+    val groupOperator = new GlobalMaterializedGroupOperator(dataSetType[Out], groupedDataSetType[Out])
+    this.connectTo(groupOperator, 0)
+    DataQuantaHackit.wrap[JavaIterable[Out]](groupOperator)
+  }
+
+  /**
+   * Feed this instance and a further instance into a [[UnionAllOperator]].
+   *
+   * @param that the other instance to union with
+   * @return a new instance representing the [[UnionAllOperator]]'s output
+   */
+  override def union(that: DataQuanta[Out]): DataQuantaHackit[Out] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val unionAllOperator = new UnionAllOperator(dataSetType[Out])
+    this.connectTo(unionAllOperator, 0)
+    that.connectTo(unionAllOperator, 1)
+    DataQuantaHackit.wrap[Out](unionAllOperator)
+  }
+
+  /**
+   * Feed this instance and a further instance into a [[IntersectOperator]].
+   *
+   * @param that the other instance to intersect with
+   * @return a new instance representing the [[IntersectOperator]]'s output
+   */
+  override def intersect(that: DataQuanta[Out]): DataQuantaHackit[Out] =  {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val intersectOperator = new IntersectOperator(dataSetType[Out])
+    this.connectTo(intersectOperator, 0)
+    that.connectTo(intersectOperator, 1)
+    DataQuantaHackit.wrap[Out](intersectOperator)
+  }
+
+  /**
+   * Feeds this and a further instance into a [[CartesianOperator]].
+   *
+   * @param that the other instance
+   * @return a new instance representing the [[CartesianOperator]]'s output
+   */
+  override def cartesian[ThatOut: ClassTag](that: DataQuanta[ThatOut]): DataQuantaHackit[WayangTuple2[Out, ThatOut]] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val cartesianOperator = new CartesianOperator(dataSetType[Out], dataSetType[ThatOut])
+    this.connectTo(cartesianOperator, 0)
+    that.connectTo(cartesianOperator, 1)
+    DataQuantaHackit.wrap[WayangTuple2[Out, ThatOut]](cartesianOperator)
+  }
+
+  /**
+   * Feeds this instance into a [[ZipWithIdOperator]].
+   *
+   * @return a new instance representing the [[ZipWithIdOperator]]'s output
+   */
+  override def zipWithId: DataQuantaHackit[WayangTuple2[lang.Long, Out]] = {
+    val zipWithIdOperator = new ZipWithIdOperator(dataSetType[Out])
+    this.connectTo(zipWithIdOperator, 0)
+    DataQuantaHackit.wrap[WayangTuple2[lang.Long, Out]](zipWithIdOperator)
+  }
+
+  /**
+   * Feeds this instance into a [[DistinctOperator]].
+   *
+   * @return a new instance representing the [[DistinctOperator]]'s output
+   */
+  override def distinct: DataQuantaHackit[Out] = {
+    val distinctOperator = new DistinctOperator(dataSetType[Out])
+    this.connectTo(distinctOperator, 0)
+    DataQuantaHackit.wrap[Out](distinctOperator)
+  }
+
+  /**
+   * Feeds this instance into a [[CountOperator]].
+   *
+   * @return a new instance representing the [[CountOperator]]'s output
+   */
+  override def count: DataQuantaHackit[lang.Long] = {
+    val countOperator = new CountOperator(dataSetType[Out])
+    this.connectTo(countOperator, 0)
+    DataQuantaHackit.wrap[lang.Long](countOperator)
+  }
+}
+
+object DataQuantaHackit extends DataQuantaCreator{
+
+  def wrap[T:ClassTag](operator: ElementaryOperator, outputIndex: Int = 0)(implicit planBuilder: PlanBuilder): DataQuantaHackit[T] = {
+    new DataQuantaHackit[T](operator, outputIndex)
+  }
+
+  def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuantaHackit[_] =
+    new DataQuantaHackit(output.getOwner.asInstanceOf[ElementaryOperator], output.getIndex)(ClassTag(output.getType.getDataUnitType.getTypeClass), planBuilder)
+
+}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/java/org/apache/wayang/api/JavaApiTest.java b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/java/org/apache/wayang/api/JavaApiTest.java
deleted file mode 100644
index aaf1f16..0000000
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/java/org/apache/wayang/api/JavaApiTest.java
+++ /dev/null
@@ -1,711 +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.wayang.api;
-
-import org.apache.wayang.core.api.WayangContext;
-import org.apache.wayang.core.util.WayangCollections;
-import org.apache.wayang.java.Java;
-//import org.apache.wayang.spark.Spark;
-//import org.apache.wayang.sqlite3.Sqlite3;
-//import org.apache.wayang.sqlite3.operators.Sqlite3TableSource;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * Test suite for the Java API.
- */
-public class JavaApiTest {
-
-//    private Configuration sqlite3Configuration;
-//
-//    @Before
-//    public void setUp() throws SQLException, IOException {
-//        // Generate test data.
-//        this.sqlite3Configuration = new Configuration();
-//        File sqlite3dbFile = File.createTempFile("wayang-sqlite3", "db");
-//        sqlite3dbFile.deleteOnExit();
-//        this.sqlite3Configuration.setProperty("wayang.sqlite3.jdbc.url", "jdbc:sqlite:" + sqlite3dbFile.getAbsolutePath());
-//        try (Connection connection = Sqlite3.platform().createDatabaseDescriptor(this.sqlite3Configuration).createJdbcConnection()) {
-//            Statement statement = connection.createStatement();
-//            statement.addBatch("DROP TABLE IF EXISTS customer;");
-//            statement.addBatch("CREATE TABLE customer (name TEXT, age INT);");
-//            statement.addBatch("INSERT INTO customer VALUES ('John', 20)");
-//            statement.addBatch("INSERT INTO customer VALUES ('Timmy', 16)");
-//            statement.addBatch("INSERT INTO customer VALUES ('Evelyn', 35)");
-//            statement.executeBatch();
-//        }
-//    }
-
-    @Test
-    public void testMapReduce() {
-        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
-
-        List<Integer> inputCollection = Arrays.asList(0, 1, 2, 3, 4);
-        Collection<Integer> outputCollection = javaPlanBuilder
-                .loadCollection(inputCollection).withName("load numbers")
-                .map(i -> i * i).withName("square")
-                .reduce((a, b) -> a + b).withName("sum")
-                .collect();
-
-        Assert.assertEquals(WayangCollections.asSet(1 + 4 + 9 + 16), WayangCollections.asSet(outputCollection));
-    }
-
-//    @Test
-//    public void testMapReduceBy() {
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
-//
-//        List<Integer> inputCollection = Arrays.asList(0, 1, 2, 3, 4);
-//        Collection<Integer> outputCollection = javaPlanBuilder
-//                .loadCollection(inputCollection).withName("load numbers")
-//                .map(i -> i * i).withName("square")
-//                .reduceByKey(i -> i & 1, (a, b) -> a + b).withName("sum")
-//                .collect();
-//
-//        Assert.assertEquals(WayangCollections.asSet(4 + 16, 1 + 9), WayangCollections.asSet(outputCollection));
-//    }
-//
-//    @Test
-//    public void testBroadcast2() {
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
-//
-//        List<Integer> inputCollection = Arrays.asList(0, 1, 2, 3, 4);
-//        List<Integer> offsetCollection = Collections.singletonList(-2);
-//
-//        LoadCollectionDataQuantaBuilder<Integer> offsetDataQuanta = javaPlanBuilder
-//                .loadCollection(offsetCollection)
-//                .withName("load offset");
-//
-//        Collection<Integer> outputCollection = javaPlanBuilder
-//                .loadCollection(inputCollection).withName("load numbers")
-//                .map(new AddOffset("offset")).withName("add offset").withBroadcast(offsetDataQuanta, "offset")
-//                .collect();
-//
-//        Assert.assertEquals(WayangCollections.asSet(-2, -1, 0, 1, 2), WayangCollections.asSet(outputCollection));
-//    }
-//
-//    @Test
-//    public void testCustomOperatorShortCut() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//
-//        final List<Integer> inputValues = WayangArrays.asList(0, 1, 2, 3);
-//
-//        // Build and execute a Wayang plan.
-//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
-//                .loadCollection(inputValues).withName("Load input values")
-//                .<Integer>customOperator(new JavaMapOperator<>(
-//                        DataSetType.createDefault(Integer.class),
-//                        DataSetType.createDefault(Integer.class),
-//                        new TransformationDescriptor<>(
-//                                i -> i + 2,
-//                                Integer.class, Integer.class
-//                        )
-//                )).withName("Add 2")
-//                .collect();
-//
-//        // Check the outcome.
-//        final List<Integer> expectedOutputValues = WayangArrays.asList(2, 3, 4, 5);
-//        Assert.assertEquals(WayangCollections.asSet(expectedOutputValues), WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testWordCount() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//
-//        final List<String> inputValues = Arrays.asList("Big data is big.", "Is data big data?");
-//
-//        // Build and execute a Wayang plan.
-//        final Collection<Tuple2<String, Integer>> outputValues = new JavaPlanBuilder(wayangContext)
-//                .loadCollection(inputValues).withName("Load input values")
-//                .flatMap(line -> Arrays.asList(line.split("\\s+"))).withName("Split words")
-//                .map(token -> token.replaceAll("\\W+", "").toLowerCase()).withName("To lower case")
-//                .map(word -> new Tuple2<>(word, 1)).withName("Attach counter")
-//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.field0, t1.field1 + t2.field1)).withName("Sum counters")
-//                .collect();
-//
-//        // Check the outcome.
-//        final Set<Tuple2<String, Integer>> expectedOutputValues = WayangCollections.asSet(
-//                new Tuple2<>("big", 3),
-//                new Tuple2<>("is", 2),
-//                new Tuple2<>("data", 3)
-//        );
-//        Assert.assertEquals(WayangCollections.asSet(expectedOutputValues), WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testWordCountOnSparkAndJava() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
-//
-//        final List<String> inputValues = Arrays.asList("Big data is big.", "Is data big data?");
-//
-//        // Build and execute a Wayang plan.
-//        final Collection<Tuple2<String, Integer>> outputValues = new JavaPlanBuilder(wayangContext)
-//                .loadCollection(inputValues).withName("Load input values")
-//                .flatMap(line -> Arrays.asList(line.split("\\s+"))).withName("Split words")
-//                .map(token -> token.replaceAll("\\W+", "").toLowerCase()).withName("To lower case")
-//                .map(word -> new Tuple2<>(word, 1)).withName("Attach counter")
-//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.field0, t1.field1 + t2.field1)).withName("Sum counters")
-//                .collect();
-//
-//        // Check the outcome.
-//        final Set<Tuple2<String, Integer>> expectedOutputValues = WayangCollections.asSet(
-//                new Tuple2<>("big", 3),
-//                new Tuple2<>("is", 2),
-//                new Tuple2<>("data", 3)
-//        );
-//        Assert.assertEquals(WayangCollections.asSet(expectedOutputValues), WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testSample() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
-//
-//        // Create some input values.
-//        final List<Integer> inputValues = WayangArrays.asList(WayangArrays.range(100));
-//
-//        // Build and execute a Wayang plan.
-//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
-//                .loadCollection(inputValues).withName("Load input values")
-//                .sample(10).withName("Sample")
-//                .collect();
-//
-//        // Check the outcome.
-//        Assert.assertEquals(10, outputValues.size());
-//        Assert.assertEquals(10, WayangCollections.asSet(outputValues).size());
-//
-//    }
-//
-//    @Test
-//    public void testDoWhile() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//
-//        // Generate test data.
-//        final List<Integer> inputValues = WayangArrays.asList(1, 2);
-//
-//        // Build and execute a word count WayangPlan.
-//
-//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
-//                .loadCollection(inputValues).withName("Load input values")
-//                .doWhile(
-//                        values -> values.stream().mapToInt(i -> i).sum() > 100,
-//                        start -> {
-//                            final GlobalReduceDataQuantaBuilder<Integer> sum =
-//                                    start.reduce((a, b) -> a + b).withName("sum");
-//                            return new Tuple<>(
-//                                    start.union(sum).withName("Old+new"),
-//                                    sum
-//                            );
-//                        }
-//                ).withConditionClass(Integer.class).withName("While <= 100")
-//                .collect();
-//
-//        Set<Integer> expectedValues = WayangCollections.asSet(1, 2, 3, 6, 12, 24, 48, 96, 192);
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    private static class AddOffset implements FunctionDescriptor.ExtendedSerializableFunction<Integer, Integer> {
-//
-//        private final String broadcastName;
-//
-//        private int offset;
-//
-//        public AddOffset(String broadcastName) {
-//            this.broadcastName = broadcastName;
-//        }
-//
-//        @Override
-//        public void open(ExecutionContext ctx) {
-//            this.offset = WayangCollections.getSingle(ctx.<Integer>getBroadcast(this.broadcastName));
-//        }
-//
-//        @Override
-//        public Integer apply(Integer input) {
-//            return input + this.offset;
-//        }
-//    }
-//
-//    @Test
-//    public void testRepeat() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//
-//        // Generate test data.
-//        final List<Integer> inputValues = WayangArrays.asList(1, 2);
-//
-//        // Build and execute a word count WayangPlan.
-//
-//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
-//                .loadCollection(inputValues).withName("Load input values")
-//                .repeat(3, start -> start
-//                        .reduce((a, b) -> a * b).withName("Multiply")
-//                        .flatMap(v -> Arrays.asList(v, v + 1)).withName("Duplicate").withOutputClass(Integer.class)
-//                ).withName("Repeat 3x")
-//                .collect();
-//
-//        Set<Integer> expectedValues = WayangCollections.asSet(42, 43);
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    private static class SelectWords implements PredicateDescriptor.ExtendedSerializablePredicate<String> {
-//
-//        private final String broadcastName;
-//
-//        private Collection<Character> selectors;
-//
-//        public SelectWords(String broadcastName) {
-//            this.broadcastName = broadcastName;
-//        }
-//
-//        @Override
-//        public void open(ExecutionContext ctx) {
-//            this.selectors = ctx.getBroadcast(this.broadcastName);
-//        }
-//
-//        @Override
-//        public boolean test(String word) {
-//            return this.selectors.stream().anyMatch(c -> word.indexOf(c) >= 0);
-//        }
-//    }
-//
-//    @Test
-//    public void testBroadcast() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<String> inputValues = Arrays.asList("Hello", "World", "Hi", "Mars");
-//        final List<Character> selectors = Arrays.asList('o', 'l');
-//
-//        // Execute the job.
-//        final DataQuantaBuilder<?, Character> selectorsDataSet = builder.loadCollection(selectors).withName("Load selectors");
-//        final Collection<String> outputValues = builder
-//                .loadCollection(inputValues).withName("Load input values")
-//                .filter(new SelectWords("selectors")).withName("Filter words")
-//                .withBroadcast(selectorsDataSet, "selectors")
-//                .collect();
-//
-//        // Verify the outcome.
-//        Set<String> expectedValues = WayangCollections.asSet("Hello", "World");
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testGroupBy() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Integer> inputValues = Arrays.asList(1, 2, 3, 4, 5, 7, 8, 9, 10);
-//
-//        // Execute the job.
-//        final Collection<Double> outputValues = builder
-//                .loadCollection(inputValues).withName("Load input values")
-//                .groupByKey(i -> i % 2).withName("group odd and even")
-//                .map(group -> {
-//                    List<Integer> sortedGroup = StreamSupport.stream(group.spliterator(), false)
-//                            .sorted()
-//                            .collect(Collectors.toList());
-//                    int sizeDivTwo = sortedGroup.size() / 2;
-//                    return sortedGroup.size() % 2 == 0 ?
-//                            (sortedGroup.get(sizeDivTwo - 1) + sortedGroup.get(sizeDivTwo)) / 2d :
-//                            (double) sortedGroup.get(sizeDivTwo);
-//                })
-//                .collect();
-//
-//        // Verify the outcome.
-//        Set<Double> expectedValues = WayangCollections.asSet(5d, 6d);
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testJoin() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
-//                new Tuple2<>("Water", 0),
-//                new Tuple2<>("Tonic", 5),
-//                new Tuple2<>("Juice", 10)
-//        );
-//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
-//                new Tuple2<>("Apple juice", "Juice"),
-//                new Tuple2<>("Tap water", "Water"),
-//                new Tuple2<>("Orange juice", "Juice")
-//        );
-//
-//        // Execute the job.
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
-//        final Collection<Tuple2<String, Integer>> outputValues = dataQuanta1
-//                .join(Tuple2::getField0, dataQuanta2, Tuple2::getField1)
-//                .map(joinTuple -> new Tuple2<>(joinTuple.getField1().getField0(), joinTuple.getField0().getField1()))
-//                .collect();
-//
-//        // Verify the outcome.
-//        Set<Tuple2<String, Integer>> expectedValues = WayangCollections.asSet(
-//                new Tuple2<>("Apple juice", 10),
-//                new Tuple2<>("Orange juice", 10),
-//                new Tuple2<>("Tap water", 0)
-//        );
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testJoinAndAssemble() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
-//                new Tuple2<>("Water", 0),
-//                new Tuple2<>("Tonic", 5),
-//                new Tuple2<>("Juice", 10)
-//        );
-//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
-//                new Tuple2<>("Apple juice", "Juice"),
-//                new Tuple2<>("Tap water", "Water"),
-//                new Tuple2<>("Orange juice", "Juice")
-//        );
-//
-//        // Execute the job.
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
-//        final Collection<Tuple2<String, Integer>> outputValues = dataQuanta1.keyBy(Tuple2::getField0)
-//                .join(dataQuanta2.keyBy(Tuple2::getField1))
-//                .assemble((val1, val2) -> new Tuple2<>(val2.getField0(), val1.getField1()))
-//                .collect();
-//
-//        // Verify the outcome.
-//        Set<Tuple2<String, Integer>> expectedValues = WayangCollections.asSet(
-//                new Tuple2<>("Apple juice", 10),
-//                new Tuple2<>("Orange juice", 10),
-//                new Tuple2<>("Tap water", 0)
-//        );
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testCoGroup() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
-//                new Tuple2<>("Water", 0),
-//                new Tuple2<>("Cola", 5),
-//                new Tuple2<>("Juice", 10)
-//        );
-//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
-//                new Tuple2<>("Apple juice", "Juice"),
-//                new Tuple2<>("Tap water", "Water"),
-//                new Tuple2<>("Orange juice", "Juice")
-//        );
-//
-//        // Execute the job.
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
-//        final Collection<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> outputValues = dataQuanta1
-//                .coGroup(Tuple2::getField0, dataQuanta2, Tuple2::getField1)
-//                .map(joinTuple -> new Tuple2<>(
-//                        WayangCollections.asSet(joinTuple.getField0()),
-//                        WayangCollections.asSet(joinTuple.getField1())
-//                ))
-//                .collect();
-//
-//        // Verify the outcome.
-//        Set<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> expectedValues = WayangCollections.asSet(
-//                new Tuple2<>(
-//                        WayangCollections.asSet(new Tuple2<>("Water", 0)),
-//                        WayangCollections.asSet(new Tuple2<>("Tap water", "Water"))
-//                ),
-//                new Tuple2<>(
-//                        WayangCollections.asSet(new Tuple2<>("Cola", 5)),
-//                        WayangCollections.asSet()
-//                ), new Tuple2<>(
-//                        WayangCollections.asSet(new Tuple2<>("Juice", 10)),
-//                        WayangCollections.asSet(new Tuple2<>("Apple juice", "Juice"), new Tuple2<>("Orange juice", "Juice"))
-//                )
-//        );
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testCoGroupViaKeyBy() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
-//                new Tuple2<>("Water", 0),
-//                new Tuple2<>("Cola", 5),
-//                new Tuple2<>("Juice", 10)
-//        );
-//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
-//                new Tuple2<>("Apple juice", "Juice"),
-//                new Tuple2<>("Tap water", "Water"),
-//                new Tuple2<>("Orange juice", "Juice")
-//        );
-//
-//        // Execute the job.
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
-//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
-//        final Collection<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> outputValues =
-//                dataQuanta1.keyBy(Tuple2::getField0)
-//                        .coGroup(dataQuanta2.keyBy(Tuple2::getField1))
-//                        .map(joinTuple -> new Tuple2<>(
-//                                WayangCollections.asSet(joinTuple.getField0()),
-//                                WayangCollections.asSet(joinTuple.getField1())
-//                        ))
-//                        .collect();
-//
-//        // Verify the outcome.
-//        Set<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> expectedValues = WayangCollections.asSet(
-//                new Tuple2<>(
-//                        WayangCollections.asSet(new Tuple2<>("Water", 0)),
-//                        WayangCollections.asSet(new Tuple2<>("Tap water", "Water"))
-//                ),
-//                new Tuple2<>(
-//                        WayangCollections.asSet(new Tuple2<>("Cola", 5)),
-//                        WayangCollections.asSet()
-//                ), new Tuple2<>(
-//                        WayangCollections.asSet(new Tuple2<>("Juice", 10)),
-//                        WayangCollections.asSet(new Tuple2<>("Apple juice", "Juice"), new Tuple2<>("Orange juice", "Juice"))
-//                )
-//        );
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testIntersect() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Integer> inputValues1 = Arrays.asList(1, 2, 3, 4, 5, 7, 8, 9, 10);
-//        final List<Integer> inputValues2 = Arrays.asList(0, 2, 3, 3, 4, 5, 7, 8, 9, 11);
-//
-//        // Execute the job.
-//        final LoadCollectionDataQuantaBuilder<Integer> dataQuanta1 = builder.loadCollection(inputValues1);
-//        final LoadCollectionDataQuantaBuilder<Integer> dataQuanta2 = builder.loadCollection(inputValues2);
-//        final Collection<Integer> outputValues = dataQuanta1.intersect(dataQuanta2).collect();
-//
-//        // Verify the outcome.
-//        Set<Integer> expectedValues = WayangCollections.asSet(2, 3, 4, 5, 7, 8, 9);
-//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testSort() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        final List<Integer> inputValues1 = Arrays.asList(3, 4, 5, 2, 1);
-//
-//        // Execute the job.
-//        final LoadCollectionDataQuantaBuilder<Integer> dataQuanta1 = builder.loadCollection(inputValues1);
-//        final Collection<Integer> outputValues = dataQuanta1.sort(r -> r).collect();
-//
-//        // Verify the outcome.
-//        List<Integer> expectedValues = Arrays.asList(1, 2, 3, 4, 5);
-//        Assert.assertEquals(expectedValues, WayangCollections.asList(outputValues));
-//    }
-//
-//
-//    @Test
-//    public void testPageRank() {
-//        // Set up WayangContext.
-//        WayangContext wayangContext = new WayangContext()
-//                .with(Java.basicPlugin())
-//                .with(Java.graphPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Create a test graph.
-//        Collection<Tuple2<Long, Long>> edges = Arrays.asList(
-//                new Tuple2<>(0L, 1L),
-//                new Tuple2<>(0L, 2L),
-//                new Tuple2<>(0L, 3L),
-//                new Tuple2<>(1L, 0L),
-//                new Tuple2<>(2L, 1L),
-//                new Tuple2<>(3L, 2L),
-//                new Tuple2<>(3L, 1L)
-//        );
-//
-//        // Execute the job.
-//        Collection<Tuple2<Long, Float>> pageRanks = builder.loadCollection(edges).asEdges()
-//                .pageRank(20)
-//                .collect();
-//        List<Tuple2<Long, Float>> sortedPageRanks = new ArrayList<>(pageRanks);
-//        sortedPageRanks.sort((pr1, pr2) -> Float.compare(pr2.field1, pr1.field1));
-//
-//        System.out.println(sortedPageRanks);
-//        Assert.assertEquals(1L, sortedPageRanks.get(0).field0.longValue());
-//        Assert.assertEquals(0L, sortedPageRanks.get(1).field0.longValue());
-//        Assert.assertEquals(2L, sortedPageRanks.get(2).field0.longValue());
-//        Assert.assertEquals(3L, sortedPageRanks.get(3).field0.longValue());
-//    }
-//
-//    @Test
-//    public void testMapPartitions() {
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        List<Integer> inputValues = WayangArrays.asList(0, 1, 2, 3, 4, 6, 8);
-//
-//        // Execute the job.
-//        Collection<Tuple2<String, Integer>> outputValues = builder.loadCollection(inputValues)
-//                .mapPartitions(partition -> {
-//                    int numEvens = 0, numOdds = 0;
-//                    for (Integer value : partition) {
-//                        if ((value & 1) == 0) numEvens++;
-//                        else numOdds++;
-//                    }
-//                    return Arrays.asList(
-//                            new Tuple2<>("odd", numOdds),
-//                            new Tuple2<>("even", numEvens)
-//                    );
-//                })
-//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.getField0(), t1.getField1() + t2.getField1()))
-//                .collect();
-//
-//        // Check the output.
-//        Set<Tuple2<String, Integer>> expectedOutput = WayangCollections.asSet(
-//                new Tuple2<>("even", 5), new Tuple2<>("odd", 2)
-//        );
-//        Assert.assertEquals(expectedOutput, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testZipWithId() {
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        List<Integer> inputValues = new ArrayList<>(42 * 100);
-//        for (int i = 0; i < 100; i++) {
-//            for (int j = 0; j < 42; j++) {
-//                inputValues.add(i);
-//            }
-//        }
-//
-//        // Execute the job.
-//        Collection<Tuple2<Integer, Integer>> outputValues = builder.loadCollection(inputValues)
-//                .zipWithId()
-//                .groupByKey(Tuple2::getField1)
-//                .map(group -> {
-//                    int distinctIds = (int) StreamSupport.stream(group.spliterator(), false)
-//                            .map(Tuple2::getField0)
-//                            .distinct()
-//                            .count();
-//                    return new Tuple2<>(distinctIds, 1);
-//                })
-//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.getField0(), t1.getField1() + t2.getField1()))
-//                .collect();
-//
-//        // Check the output.
-//        Set<Tuple2<Integer, Integer>> expectedOutput = Collections.singleton(new Tuple2<>(42, 100));
-//        Assert.assertEquals(expectedOutput, WayangCollections.asSet(outputValues));
-//    }
-//
-//    @Test
-//    public void testWriteTextFile() throws IOException, URISyntaxException {
-//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
-//
-//        // Generate test data.
-//        List<Double> inputValues = Arrays.asList(0d, 1 / 3d, 2 / 3d, 1d, 4 / 3d, 5 / 3d);
-//
-//        // Execute the job.
-//        File tempDir = LocalFileSystem.findTempDir();
-//        String targetUrl = LocalFileSystem.toURL(new File(tempDir, "testWriteTextFile.txt"));
-//
-//        builder
-//                .loadCollection(inputValues)
-//                .writeTextFile(targetUrl, d -> String.format("%.2f", d), "testWriteTextFile()");
-//
-//        // Check the output.
-//        Set<String> actualLines = Files.lines(Paths.get(new URI(targetUrl))).collect(Collectors.toSet());
-//        Set<String> expectedLines = inputValues.stream().map(d -> String.format("%.2f", d)).collect(Collectors.toSet());
-//        Assert.assertEquals(expectedLines, actualLines);
-//    }
-//
-//    @Test
-//    public void testSqlOnJava() throws IOException, SQLException {
-//        // Execute job.
-//        final WayangContext wayangCtx = new WayangContext(this.sqlite3Configuration)
-//                .with(Java.basicPlugin())
-//                .with(Sqlite3.plugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangCtx, "testSqlOnJava()");
-//        final Collection<String> outputValues = builder
-//                .readTable(new Sqlite3TableSource("customer", "name", "age"))
-//                .filter(r -> (Integer) r.getField(1) >= 18).withSqlUdf("age >= 18").withTargetPlatform(Java.platform())
-//                .asRecords().projectRecords(new String[]{"name"})
-//                .map(record -> (String) record.getField(0))
-//                .collect();
-//
-//        // Test the outcome.
-//        Assert.assertEquals(
-//                WayangCollections.asSet("John", "Evelyn"),
-//                WayangCollections.asSet(outputValues)
-//        );
-//    }
-//
-//    @Test
-//    public void testSqlOnSqlite3() throws IOException, SQLException {
-//        // Execute job.
-//        final WayangContext wayangCtx = new WayangContext(this.sqlite3Configuration)
-//                .with(Java.basicPlugin())
-//                .with(Sqlite3.plugin());
-//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangCtx, "testSqlOnSqlite3()");
-//        final Collection<String> outputValues = builder
-//                .readTable(new Sqlite3TableSource("customer", "name", "age"))
-//                .filter(r -> (Integer) r.getField(1) >= 18).withSqlUdf("age >= 18")
-//                .asRecords().projectRecords(new String[]{"name"}).withTargetPlatform(Sqlite3.platform())
-//                .map(record -> (String) record.getField(0))
-//                .collect();
-//
-//        // Test the outcome.
-//        Assert.assertEquals(
-//                WayangCollections.asSet("John", "Evelyn"),
-//                WayangCollections.asSet(outputValues)
-//        );
-//    }
-
-}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/api/ApiTest.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/api/ApiTest.scala
deleted file mode 100644
index ed1a7ac..0000000
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/api/ApiTest.scala
+++ /dev/null
@@ -1,575 +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.wayang.api
-
-import org.apache.wayang.basic.WayangBasics
-import org.apache.wayang.core.api.{Configuration, WayangContext}
-import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializablePredicate
-import org.apache.wayang.core.function.{ExecutionContext, TransformationDescriptor}
-import org.apache.wayang.core.util.fs.LocalFileSystem
-import org.apache.wayang.java.Java
-import org.junit.{Assert, Test}
-
-import java.io.File
-import java.net.URI
-import java.nio.file.{Files, Paths}
-import java.sql.{Connection, Statement}
-import java.util.function.Consumer
-
-/**
-  * Tests the Wayang API.
-  */
-class ApiTest {
-
-  @Test
-  def testReadMapCollect(): Unit = {
-    // Set up WayangContext.
-    val wayangContext = new WayangContext()
-                         .withPlugin(Java.basicPlugin)
-//                         .withPlugin(Spark.basicPlugin)
-    // Generate some test data.
-    val inputValues = (for (i <- 1 to 10) yield i).toArray
-
-    // Build and execute a Wayang plan.
-    val outputValues = wayangContext
-      .loadCollection(inputValues).withName("Load input values")
-      .map(_ + 2).withName("Add 2")
-      .collect()
-
-    // Check the outcome.
-    val expectedOutputValues = inputValues.map(_ + 2)
-    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
-  }
-
-//  @Test
-//  def testCustomOperator(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = (for (i <- 1 to 10) yield i).toArray
-//
-//    // Build and execute a Wayang plan.
-//    val inputDataSet = wayang.loadCollection(inputValues).withName("Load input values")
-//
-//    // Add the custom operator.
-//    val IndexedSeq(addedValues) = wayang.customOperator(new JavaMapOperator(
-//      dataSetType[Int],
-//      dataSetType[Int],
-//      new TransformationDescriptor(
-//        toSerializableFunction[Int, Int](_ + 2),
-//        basicDataUnitType[Int], basicDataUnitType[Int]
-//      )
-//    ), inputDataSet)
-//    addedValues.withName("Add 2")
-//
-//    // Collect the result.
-//    val outputValues = addedValues.asInstanceOf[DataQuanta[Int]].collect()
-//
-//    // Check the outcome.
-//    val expectedOutputValues = inputValues.map(_ + 2)
-//    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
-//  }
-//
-//  @Test
-//  def testCustomOperatorShortCut(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = (for (i <- 1 to 10) yield i).toArray
-//
-//    // Build and execute a Wayang plan.
-//    val outputValues = wayang
-//      .loadCollection(inputValues).withName("Load input values")
-//      .customOperator[Int](new JavaMapOperator(
-//      dataSetType[Int],
-//      dataSetType[Int],
-//      new TransformationDescriptor(
-//        toSerializableFunction[Int, Int](_ + 2),
-//        basicDataUnitType[Int], basicDataUnitType[Int]
-//      )
-//    )).withName("Add 2")
-//      .collect()
-//
-//    // Check the outcome.
-//    val expectedOutputValues = inputValues.map(_ + 2)
-//    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
-//  }
-//
-//  @Test
-//  def testWordCount(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = Array("Big data is big.", "Is data big data?")
-//
-//    // Build and execute a word count WayangPlan.
-//    val wordCounts = wayang
-//      .loadCollection(inputValues).withName("Load input values")
-//      .flatMap(_.split("\\s+")).withName("Split words")
-//      .map(_.replaceAll("\\W+", "").toLowerCase).withName("To lowercase")
-//      .map((_, 1)).withName("Attach counter")
-//      .reduceByKey(_._1, (a, b) => (a._1, a._2 + b._2)).withName("Sum counters")
-//      .collect().toSet
-//
-//    val expectedWordCounts = Set(("big", 3), ("is", 2), ("data", 3))
-//
-//    Assert.assertEquals(expectedWordCounts, wordCounts)
-//  }
-//
-//  @Test
-//  def testWordCountOnSparkAndJava(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = Array("Big data is big.", "Is data big data?")
-//
-//    // Build and execute a word count WayangPlan.
-//    val wordCounts = wayang
-//      .loadCollection(inputValues).withName("Load input values").withTargetPlatforms(Java.platform)
-//      .flatMap(_.split("\\s+")).withName("Split words").withTargetPlatforms(Java.platform)
-//      .map(_.replaceAll("\\W+", "").toLowerCase).withName("To lowercase").withTargetPlatforms(Spark.platform)
-//      .map((_, 1)).withName("Attach counter").withTargetPlatforms(Spark.platform)
-//      .reduceByKey(_._1, (a, b) => (a._1, a._2 + b._2)).withName("Sum counters").withTargetPlatforms(Spark.platform)
-//      .collect().toSet
-//
-//    val expectedWordCounts = Set(("big", 3), ("is", 2), ("data", 3))
-//
-//    Assert.assertEquals(expectedWordCounts, wordCounts)
-//  }
-//
-//  @Test
-//  def testSample(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = for (i <- 0 until 100) yield i
-//
-//    // Build and execute the WayangPlan.
-//    val sample = wayang
-//      .loadCollection(inputValues)
-//      .sample(10)
-//      .collect()
-//
-//    // Check the result.
-//    Assert.assertEquals(10, sample.size)
-//    Assert.assertEquals(10, sample.toSet.size)
-//  }
-//
-//  @Test
-//  def testDoWhile(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = Array(1, 2)
-//
-//    // Build and execute a word count WayangPlan.
-//
-//    val values = wayang
-//      .loadCollection(inputValues).withName("Load input values")
-//      .doWhile[Int](vals => vals.max > 100, {
-//      start =>
-//        val sum = start.reduce(_ + _).withName("Sum")
-//        (start.union(sum).withName("Old+new"), sum)
-//    }).withName("While <= 100")
-//      .collect().toSet
-//
-//    val expectedValues = Set(1, 2, 3, 6, 12, 24, 48, 96, 192)
-//    Assert.assertEquals(expectedValues, values)
-//  }
-//
-//  @Test
-//  def testRepeat(): Unit = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    // Generate some test data.
-//    val inputValues = Array(1, 2)
-//
-//    // Build and execute a word count WayangPlan.
-//
-//    val values = wayang
-//      .loadCollection(inputValues).withName("Load input values").withName(inputValues.mkString(","))
-//      .repeat(3,
-//        _.reduce(_ * _).withName("Multiply")
-//          .flatMap(v => Seq(v, v + 1)).withName("Duplicate")
-//      ).withName("Repeat 3x")
-//      .collect().toSet
-//
-//    // initial: 1,2 -> 1st: 2,3 -> 2nd: 6,7 => 3rd: 42,43
-//    val expectedValues = Set(42, 43)
-//    Assert.assertEquals(expectedValues, values)
-//  }
-//
-//  @Test
-//  def testBroadcast() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//    val builder = new PlanBuilder(wayang)
-//
-//    // Generate some test data.
-//    val inputStrings = Array("Hello", "World", "Hi", "Mars")
-//    val selectors = Array('o', 'l')
-//
-//    val selectorsDataSet = builder.loadCollection(selectors).withName("Load selectors")
-//
-//    // Build and execute a word count WayangPlan.
-//    val values = builder
-//      .loadCollection(inputStrings).withName("Load input values")
-//      .filterJava(new ExtendedSerializablePredicate[String] {
-//
-//        var selectors: Iterable[Char] = _
-//
-//        override def open(ctx: ExecutionContext): Unit = {
-//          import scala.collection.JavaConversions._
-//          selectors = collectionAsScalaIterable(ctx.getBroadcast[Char]("selectors"))
-//        }
-//
-//        override def test(t: String): Boolean = selectors.forall(selector => t.contains(selector))
-//
-//      }).withName("Filter words")
-//      .withBroadcast(selectorsDataSet, "selectors")
-//      .collect().toSet
-//
-//    val expectedValues = Set("Hello", "World")
-//    Assert.assertEquals(expectedValues, values)
-//  }
-//
-//  @Test
-//  def testGroupBy() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues = Array(1, 2, 3, 4, 5, 7, 8, 9, 10)
-//
-//    val result = wayang
-//      .loadCollection(inputValues)
-//      .groupByKey(_ % 2).withName("group odd and even")
-//      .map {
-//        group =>
-//          import scala.collection.JavaConversions._
-//          val buffer = group.toBuffer
-//          buffer.sortBy(identity)
-//          if (buffer.size % 2 == 0) (buffer(buffer.size / 2 - 1) + buffer(buffer.size / 2)) / 2
-//          else buffer(buffer.size / 2)
-//      }.withName("median")
-//      .collect()
-//
-//    val expectedValues = Set(5, 6)
-//    Assert.assertEquals(expectedValues, result.toSet)
-//  }
-//
-//  @Test
-//  def testGroup() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues = Array(1, 2, 3, 4, 5, 7, 8, 9, 10)
-//
-//    val result = wayang
-//      .loadCollection(inputValues)
-//      .group()
-//      .map {
-//        group =>
-//          import scala.collection.JavaConversions._
-//          val buffer = group.toBuffer
-//          buffer.sortBy(int => int)
-//          if (buffer.size % 2 == 0) (buffer(buffer.size / 2) + buffer(buffer.size / 2 + 1)) / 2
-//          else buffer(buffer.size / 2)
-//      }
-//      .collect()
-//
-//    val expectedValues = Set(5)
-//    Assert.assertEquals(expectedValues, result.toSet)
-//  }
-//
-//  @Test
-//  def testJoin() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues1 = Array(("Water", 0), ("Tonic", 5), ("Juice", 10))
-//    val inputValues2 = Array(("Apple juice", "Juice"), ("Tap water", "Water"), ("Orange juice", "Juice"))
-//
-//    val builder = new PlanBuilder(wayang)
-//    val dataQuanta1 = builder.loadCollection(inputValues1)
-//    val dataQuanta2 = builder.loadCollection(inputValues2)
-//    val result = dataQuanta1
-//      .join[(String, String), String](_._1, dataQuanta2, _._2)
-//      .map(joinTuple => (joinTuple.field1._1, joinTuple.field0._2))
-//      .collect()
-//
-//    val expectedValues = Set(("Apple juice", 10), ("Tap water", 0), ("Orange juice", 10))
-//    Assert.assertEquals(expectedValues, result.toSet)
-//  }
-//
-//  @Test
-//  def testJoinAndAssemble() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues1 = Array(("Water", 0), ("Tonic", 5), ("Juice", 10))
-//    val inputValues2 = Array(("Apple juice", "Juice"), ("Tap water", "Water"), ("Orange juice", "Juice"))
-//
-//    val builder = new PlanBuilder(wayang)
-//    val dataQuanta1 = builder.loadCollection(inputValues1)
-//    val dataQuanta2 = builder.loadCollection(inputValues2)
-//    val result = dataQuanta1.keyBy(_._1).join(dataQuanta2.keyBy(_._2))
-//      .assemble((dq1, dq2) => (dq2._1, dq1._2))
-//      .collect()
-//
-//    val expectedValues = Set(("Apple juice", 10), ("Tap water", 0), ("Orange juice", 10))
-//    Assert.assertEquals(expectedValues, result.toSet)
-//  }
-//
-//
-//  @Test
-//  def testCoGroup() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues1 = Array(("Water", 0), ("Cola", 5), ("Juice", 10))
-//    val inputValues2 = Array(("Apple juice", "Juice"), ("Tap water", "Water"), ("Orange juice", "Juice"))
-//
-//    val builder = new PlanBuilder(wayang)
-//    val dataQuanta1 = builder.loadCollection(inputValues1)
-//    val dataQuanta2 = builder.loadCollection(inputValues2)
-//    val result = dataQuanta1
-//      .coGroup[(String, String), String](_._1, dataQuanta2, _._2)
-//      .collect()
-//
-//    import scala.collection.JavaConversions._
-//    val actualValues = result.map(coGroup => (coGroup.field0.toSet, coGroup.field1.toSet)).toSet
-//    val expectedValues = Set(
-//      (Set(("Water", 0)), Set(("Tap water", "Water"))),
-//      (Set(("Cola", 5)), Set()),
-//      (Set(("Juice", 10)), Set(("Apple juice", "Juice"), ("Orange juice", "Juice")))
-//    )
-//    Assert.assertEquals(expectedValues, actualValues)
-//  }
-//
-//  @Test
-//  def testIntersect() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues1 = Array(1, 2, 3, 4, 5, 7, 8, 9, 10)
-//    val inputValues2 = Array(0, 2, 3, 3, 4, 5, 7, 8, 9, 11)
-//
-//    val builder = new PlanBuilder(wayang)
-//    val dataQuanta1 = builder.loadCollection(inputValues1)
-//    val dataQuanta2 = builder.loadCollection(inputValues2)
-//    val result = dataQuanta1
-//      .intersect(dataQuanta2)
-//      .collect()
-//
-//    val expectedValues = Set(2, 3, 4, 5, 7, 8, 9)
-//    Assert.assertEquals(expectedValues, result.toSet)
-//  }
-//
-//
-//  @Test
-//  def testSort() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues1 = Array(3, 4, 5, 2, 1)
-//
-//    val builder = new PlanBuilder(wayang)
-//    val dataQuanta1 = builder.loadCollection(inputValues1)
-//    val result = dataQuanta1
-//      .sort(r=>r)
-//      .collect()
-//
-//    val expectedValues = Array(1, 2, 3, 4, 5)
-//    Assert.assertArrayEquals(expectedValues, result.toArray)
-//  }
-//
-//
-//  @Test
-//  def testPageRank() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext()
-//      .withPlugin(Java.graphPlugin)
-//      .withPlugin(WayangBasics.graphPlugin)
-//      .withPlugin(Java.basicPlugin)
-//    import org.apache.wayang.api.graph._
-//
-//    val edges = Seq((0, 1), (0, 2), (0, 3), (1, 0), (2, 1), (3, 2), (3, 1)).map(t => Edge(t._1, t._2))
-//
-//    val pageRanks = wayang
-//      .loadCollection(edges).withName("Load edges")
-//      .pageRank(20).withName("PageRank")
-//      .collect()
-//      .map(t => t.field0.longValue -> t.field1)
-//      .toMap
-//
-//    print(pageRanks)
-//    // Let's not check absolute numbers but only the relative ordering.
-//    Assert.assertTrue(pageRanks(1) > pageRanks(0))
-//    Assert.assertTrue(pageRanks(0) > pageRanks(2))
-//    Assert.assertTrue(pageRanks(2) > pageRanks(3))
-//  }
-//
-//  @Test
-//  def testMapPartitions() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext()
-//      .withPlugin(Java.basicPlugin())
-//      .withPlugin(Spark.basicPlugin)
-//
-//    val typeCounts = wayang
-//      .loadCollection(Seq(0, 1, 2, 3, 4, 6, 8))
-//        .mapPartitions { ints =>
-//          var (numOdds, numEvens) = (0, 0)
-//          ints.foreach(i => if ((i & 1) == 0) numEvens += 1 else numOdds += 1)
-//          Seq(("odd", numOdds), ("even", numEvens))
-//        }
-//      .reduceByKey(_._1, { case ((kind1, count1), (kind2, count2)) => (kind1, count1 + count2) })
-//      .collect()
-//
-//    Assert.assertEquals(Set(("odd", 2), ("even", 5)), typeCounts.toSet)
-//  }
-//
-//  @Test
-//  def testZipWithId() = {
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
-//
-//    val inputValues = for (i <- 0 until 100; j <- 0 until 42) yield i
-//
-//    val result = wayang
-//      .loadCollection(inputValues)
-//      .zipWithId
-//      .groupByKey(_.field1)
-//      .map { group =>
-//        import scala.collection.JavaConversions._
-//        (group.map(_.field0).toSet.size, 1)
-//      }
-//      .reduceByKey(_._1, (t1, t2) => (t1._1, t1._2 + t2._2))
-//      .collect()
-//
-//    val expectedValues = Set((42, 100))
-//    Assert.assertEquals(expectedValues, result.toSet)
-//  }
-//
-//  @Test
-//  def testWriteTextFile() = {
-//    val tempDir = LocalFileSystem.findTempDir
-//    val targetUrl = LocalFileSystem.toURL(new File(tempDir, "testWriteTextFile.txt"))
-//
-//    // Set up WayangContext.
-//    val wayang = new WayangContext().withPlugin(Java.basicPlugin)
-//
-//    val inputValues = for (i <- 0 to 5) yield i * 0.333333333333
-//
-//    val result = wayang
-//      .loadCollection(inputValues)
-//      .writeTextFile(targetUrl, formatterUdf = d => f"${d % .2f}")
-//
-//    val lines = scala.collection.mutable.Set[String]()
-//    Files.lines(Paths.get(new URI(targetUrl))).forEach(new Consumer[String] {
-//      override def accept(line: String): Unit = lines += line
-//    })
-//
-//    val expectedLines = inputValues.map(v => f"${v % .2f}").toSet
-//    Assert.assertEquals(expectedLines, lines)
-//  }
-//
-//  @Test
-//  def testSqlOnJava() = {
-//    // Initialize some test data.
-//    val configuration = new Configuration
-//    val sqlite3dbFile = File.createTempFile("wayang-sqlite3", "db")
-//    sqlite3dbFile.deleteOnExit()
-//    configuration.setProperty("wayang.sqlite3.jdbc.url", "jdbc:sqlite:" + sqlite3dbFile.getAbsolutePath)
-//
-//    try {
-//      val connection: Connection = Sqlite3.platform.createDatabaseDescriptor(configuration).createJdbcConnection
-//      try {
-//        val statement: Statement = connection.createStatement
-//        statement.addBatch("DROP TABLE IF EXISTS customer;")
-//        statement.addBatch("CREATE TABLE customer (name TEXT, age INT);")
-//        statement.addBatch("INSERT INTO customer VALUES ('John', 20)")
-//        statement.addBatch("INSERT INTO customer VALUES ('Timmy', 16)")
-//        statement.addBatch("INSERT INTO customer VALUES ('Evelyn', 35)")
-//        statement.executeBatch()
-//      } finally {
-//        if (connection != null) connection.close()
-//      }
-//    }
-//
-//    // Set up WayangContext.
-//    val wayang = new WayangContext(configuration).withPlugin(Java.basicPlugin).withPlugin(Sqlite3.plugin)
-//
-//    val result = wayang
-//      .readTable(new Sqlite3TableSource("customer", "name", "age"))
-//      .filter(r => r.getField(1).asInstanceOf[Integer] >= 18, sqlUdf = "age >= 18").withTargetPlatforms(Java.platform)
-//      .projectRecords(Seq("name"))
-//      .map(_.getField(0).asInstanceOf[String])
-//      .collect()
-//      .toSet
-//
-//    val expectedValues = Set("John", "Evelyn")
-//    Assert.assertEquals(expectedValues, result)
-//  }
-//
-//  @Test
-//  def testSqlOnSqlite3() = {
-//    // Initialize some test data.
-//    val configuration = new Configuration
-//    val sqlite3dbFile = File.createTempFile("wayang-sqlite3", "db")
-//    sqlite3dbFile.deleteOnExit()
-//    configuration.setProperty("wayang.sqlite3.jdbc.url", "jdbc:sqlite:" + sqlite3dbFile.getAbsolutePath)
-//
-//    try {
-//      val connection: Connection = Sqlite3.platform.createDatabaseDescriptor(configuration).createJdbcConnection
-//      try {
-//        val statement: Statement = connection.createStatement
-//        statement.addBatch("DROP TABLE IF EXISTS customer;")
-//        statement.addBatch("CREATE TABLE customer (name TEXT, age INT);")
-//        statement.addBatch("INSERT INTO customer VALUES ('John', 20)")
-//        statement.addBatch("INSERT INTO customer VALUES ('Timmy', 16)")
-//        statement.addBatch("INSERT INTO customer VALUES ('Evelyn', 35)")
-//        statement.executeBatch
-//      } finally {
-//        if (connection != null) connection.close()
-//      }
-//    }
-//
-//    // Set up WayangContext.
-//    val wayang = new WayangContext(configuration).withPlugin(Java.basicPlugin).withPlugin(Sqlite3.plugin)
-//
-//    val result = wayang
-//      .readTable(new Sqlite3TableSource("customer", "name", "age"))
-//      .filter(r => r.getField(1).asInstanceOf[Integer] >= 18, sqlUdf = "age >= 18")
-//      .projectRecords(Seq("name")).withTargetPlatforms(Sqlite3.platform)
-//      .map(_.getField(0).asInstanceOf[String])
-//      .collect()
-//      .toSet
-//
-//    val expectedValues = Set("John", "Evelyn")
-//    Assert.assertEquals(expectedValues, result)
-//  }
-}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala
new file mode 100644
index 0000000..59a0fd9
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala
@@ -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.wayang.plugin.hackit.api
+
+import org.apache.wayang.api.ApiTest
+import org.apache.wayang.api.dataquanta.DataQuantaFactory
+import org.junit.{BeforeClass, Test}
+import org.junit.jupiter.api.{BeforeAll, BeforeEach}
+
+class ApiExtensionTest extends ApiTest {
+
+  @BeforeEach
+  def setUp() ={
+    DataQuantaFactory.setTemplate(DataQuantaHackit);
+  }
+
+  @Test
+  override def testReadMapCollect(): Unit = {
+    DataQuantaFactory.setTemplate(DataQuantaHackit);
+
+    super.testReadMapCollect()
+  }
+
+}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/wayang.properties b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/wayang.properties
new file mode 100644
index 0000000..b296279
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/wayang.properties
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+spark.driver.host = localhost

[incubator-wayang] 04/15: [WAYANG-28] javadoc to sniffer package

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 428d3283b952393c47ab660460daf9f04bf80f06
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Mon May 10 13:59:33 2021 -0400

    [WAYANG-28] javadoc to sniffer package
---
 .../plugin/hackit/core/sniffer/HackitSniffer.java  | 80 ++++++++++++++--------
 .../plugin/hackit/core/sniffer/actor/Actor.java    |  1 +
 .../hackit/core/sniffer/inject/EmptyInjector.java  |  3 +-
 .../hackit/core/sniffer/inject/Injector.java       | 15 ++--
 .../hackit/core/sniffer/shipper/PSProtocol.java    |  3 +-
 .../hackit/core/sniffer/shipper/Shipper.java       | 60 ++++++++--------
 .../sniffer/shipper/receiver/BufferReceiver.java   | 22 +++++-
 .../sniffer/shipper/receiver/EmptyReceiver.java    |  3 +-
 .../core/sniffer/shipper/receiver/Receiver.java    | 15 ++--
 .../core/sniffer/shipper/sender/EmptySender.java   |  2 +-
 .../hackit/core/sniffer/shipper/sender/Sender.java | 13 ++--
 .../core/sniffer/sniff/CollectionTagsToSniff.java  |  2 +-
 .../core/sniffer/sniff/SingleTagToSniff.java       |  2 +-
 .../plugin/hackit/core/sniffer/sniff/Sniff.java    | 10 ++-
 14 files changed, 150 insertions(+), 81 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java
index 6740265..58e0366 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java
@@ -31,12 +31,14 @@ import java.util.Iterator;
 import java.util.function.Function;
 
 /**
+ * {@link HackitSniffer} is the one of the main function on Hackit, this function have the resposability of execute the
+ * logic of sniff the tuples and apply the logic of {@link #apply(HackitTuple)}
  *
- * @param <K>
- * @param <T>
- * @param <SentType>
- * @param <SenderObj>
- * @param <ReceiverObj>
+ * @param <K> type of key that it handle the {@link org.apache.wayang.plugin.hackit.core.tuple.header.Header}
+ * @param <T> type that wrapped by {@link HackitTuple}
+ * @param <SentType> Type of the element that it will be send out by {@link Sender}
+ * @param <SenderObj> Object class of the implementation of {@link Sender}
+ * @param <ReceiverObj>Object class of the implementation of {@link Receiver}
  */
 public class
     HackitSniffer<
@@ -54,42 +56,43 @@ public class
         Serializable {
 
     /**
-     *
+     * Indicate if the first execution or not, because some function will need that information
+     * to get instantiated
      */
     private transient boolean not_first = false;
 
     /**
-     *
+     * {@link Injector} instance that will be use by {@link HackitSniffer} as component
      */
     private Injector<HackitTuple<K, T>> hackItInjector;
 
     /**
-     *
+     * {@link Actor} instance that will be use by {@link HackitSniffer} as component
      */
     private Actor<HackitTuple<K, T>> actorFunction;
 
     /**
-     *
+     * {@link Shipper} instance that will be use by {@link HackitSniffer} as component
      */
     private Shipper<HackitTuple<K, T>, SentType, SenderObj, ReceiverObj> shipper;
 
     /**
-     *
+     * {@link Sniff} instance that will be use by {@link HackitSniffer} as component
      */
     private Sniff<HackitTuple<K, T>> hackItSniff;
 
     /**
-     *
+     * {@link Cloner} instance that will be use by {@link HackitSniffer} as component
      */
     private Cloner<HackitTuple<K, T>, SentType> hackItCloner;
 
     /**
-     *
-     * @param hackItInjector
-     * @param actorFunction
-     * @param shipper
-     * @param hackItSniff
-     * @param hackItCloner
+     * Construct with the components as parameters
+     * @param hackItInjector {@link Injector} instance that will be use by {@link HackitSniffer} as component
+     * @param actorFunction {@link Actor} instance that will be use by {@link HackitSniffer} as component
+     * @param shipper {@link Shipper} instance that will be use by {@link HackitSniffer} as component
+     * @param hackItSniff {@link Sniff} instance that will be use by {@link HackitSniffer} as component
+     * @param hackItCloner {@link Cloner} instance that will be use by {@link HackitSniffer} as component
      */
     //TODO: it may private, because need to be executed just at the creation moment
     public HackitSniffer(
@@ -115,6 +118,24 @@ public class
         this.not_first = false;
     }
 
+    /**
+     * apply contains the logic that need to be executed at each tuple that is process by the main pipeline,
+     * <ol>
+     *     <li>If is the first exection the function perform the connection between the sidecar and the main pipeline</li>
+     *     <li>Validate if the tuple need to be sniffed</li>
+     *     <li>
+     *         <ol>
+     *             <li>validate if the element have the condition to be sent out</li>
+     *             <li>The tuple is cloned </li>
+     *             <li>The Tuple is sended out by publishing it</li>
+     *         </ol>
+     *     </li>
+     *     <li>From the shipper it looks if exist new elements to be injected</li>
+     * </ol>
+     *
+     * @param ktHackItTuple
+     * @return
+     */
     @Override
     public Iterator<HackitTuple<K, T>> apply(HackitTuple<K, T> ktHackItTuple) {
         if(!this.not_first){
@@ -136,9 +157,10 @@ public class
     }
 
     /**
+     * set {@link Injector} instance that will be use by {@link HackitSniffer} as component
      *
-     * @param hackItInjector
-     * @return
+     * @param hackItInjector {@link Injector} instance
+     * @return self instance of the {@link HackitSniffer}
      */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setHackItInjector(Injector<HackitTuple<K, T>> hackItInjector) {
         this.hackItInjector = hackItInjector;
@@ -146,9 +168,10 @@ public class
     }
 
     /**
+     * set {@link Actor} instance that will be use by {@link HackitSniffer} as component
      *
-     * @param actorFunction
-     * @return
+     * @param actorFunction {@link Actor} instance
+     * @return self instance of the {@link HackitSniffer}
      */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setActorFunction(Actor<HackitTuple<K, T>> actorFunction) {
         this.actorFunction = actorFunction;
@@ -156,9 +179,10 @@ public class
     }
 
     /**
+     * set {@link Shipper} instance that will be use by {@link HackitSniffer} as component
      *
-     * @param shipper
-     * @return
+     * @param shipper {@link Shipper} instance
+     * @return self instance of the {@link HackitSniffer}
      */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setShipper(Shipper<HackitTuple<K, T>, SentType, SenderObj, ReceiverObj> shipper) {
         this.shipper = shipper;
@@ -166,9 +190,10 @@ public class
     }
 
     /**
+     * set {@link Sniff} instance that will be use by {@link HackitSniffer} as component
      *
-     * @param hackItSniff
-     * @return
+     * @param hackItSniff {@link Sniff} instance
+     * @return self instance of the {@link HackitSniffer}
      */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setHackItSniff(Sniff<HackitTuple<K, T>> hackItSniff) {
         this.hackItSniff = hackItSniff;
@@ -176,9 +201,10 @@ public class
     }
 
     /**
+     * set {@link Cloner} instance that will be use by {@link HackitSniffer} as component
      *
-     * @param hackItCloner
-     * @return
+     * @param hackItCloner {@link Cloner} instance
+     * @return self instance of the {@link HackitSniffer}
      */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setHackItCloner(Cloner<HackitTuple<K, T>, SentType> hackItCloner) {
         this.hackItCloner = hackItCloner;
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java
index 0deffd8..555c63b 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java
@@ -23,6 +23,7 @@ import java.io.Serializable;
  *
  * @param <T>
  */
+//TODO it may deleted, but validate if is need it
 public interface Actor<T> extends Serializable {
 
     /**
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java
index cacca18..ece3404 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java
@@ -25,8 +25,9 @@ import java.util.Spliterators;
 import java.util.stream.StreamSupport;
 
 /**
+ * EmptyInjector is a dummy implementation of the {@link Injector} to show how it need to looks and Inyector implementation
  *
- * @param <T>
+ * @param <T> type of the element processed.
  */
 public class EmptyInjector<T> implements Injector<T>{
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java
index fb26604..edf1bc2 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java
@@ -21,13 +21,16 @@ import java.io.Serializable;
 import java.util.Iterator;
 
 /**
+ * Injector is the component on the Sniffer that it get looking to get a {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * form outside and added on the current process
  *
- * @param <T>
+ * @param <T> type of the tuple that need to be process
  */
 public interface Injector<T> extends Serializable {
 
     /**
      *
+     *
      * @param element
      * @param iterator
      * @return
@@ -35,16 +38,18 @@ public interface Injector<T> extends Serializable {
     Iterator<T> inject(T element, Iterator<T> iterator);
 
     /**
+     * Evaluate if the <code>element</code> need to skipped or process
      *
-     * @param element
-     * @return
+     * @param element that is evaluated
+     * @return True is need to be process, False in other cases
      */
     boolean is_skip_element(T element);
 
     /**
+     * Evaluate if the <code>element</code> need to halt the job or not
      *
-     * @param element
-     * @return
+     * @param element that is evaluated
+     * @return True if the process need to be halt, False in other cases
      */
     boolean is_halt_job(T element);
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java
index 703074c..8994f52 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java
@@ -23,6 +23,7 @@ package org.apache.wayang.plugin.hackit.core.sniffer.shipper;
 public interface PSProtocol {
 
     /**
+     * Add the topics on the server and messages
      *
      * @param topic
      * @return
@@ -30,7 +31,7 @@ public interface PSProtocol {
     public PSProtocol addTopic(String... topic);
 
     /**
-     *
+     * Add places where the data need to be published or retrieved
      * @param exchange
      * @return
      */
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java
index a9881af..ad01d32 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java
@@ -24,41 +24,46 @@ import java.io.Serializable;
 import java.util.Iterator;
 
 /**
+ * Shipper is the component that it handle the reception and emision of the message from the main pipeline and sidecar
+ * pipeline to enable a smoothly connection between them.
  *
- * @param <T>
- * @param <ST>
- * @param <SenderObj>
- * @param <ReceiverObj>
+ * @param <T_IN> type of the tuple that it come from the sidecar to the main pipeline
+ * @param <T_OUT> type of the tuple that it go from the main to sidecar pipeline
+ * @param <SenderObj> type of {@link Sender} that the shipper will use
+ * @param <ReceiverObj> type of {@link Receiver} that the shipper will use
  */
-public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj extends Receiver<T>> implements Iterator<T>, Serializable {
+public abstract class Shipper<T_IN, T_OUT, SenderObj extends Sender<T_OUT>, ReceiverObj extends Receiver<T_IN>> implements Iterator<T_IN>, Serializable {
 
     /**
-     *
+     * <code>sender_instance</code> instance that have {@link Sender} implementation
      */
     protected Sender sender_instance;
 
     /**
-     *
+     * <code>receiver_instance</code> instance that have {@link Receiver} implementation
      */
     protected Receiver receiver_instance;
 
     /**
+     * Generate an instance of the {@link Sender}, it could be take it by configurations
      *
-     * @return
+     * @return {@link Sender} instance
      */
     protected abstract Sender createSenderInstance();
 
     /**
+     * Generate an instance of the {@link Receiver}, it could be take it by configurations
      *
-     * @return
+     * @return {@link Receiver} instance
      */
     protected abstract Receiver createReceiverInstance();
 
     /**
-     * Connect with the a Message queue service
-     * @param value
+     * Connect with the a Message queue service and send the message
+     *
+     * @param value is the element that it will be send out form the main pipeline
      */
-    public void publish(ST value){
+    public void publish(T_OUT value){
         if(this.sender_instance == null){
             throw new RuntimeException("The Sender of the Shipper is not instanciated");
         }
@@ -74,17 +79,16 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     }
 
     /**
-     *
-     * @param topic
+     * @see #subscribeAsProducer()
+     * @param topic list of topic where the messages need to be seeded
      */
     public void subscribeAsProducer(String... topic){
         this.subscribeAsProducer("default", topic);
     }
 
     /**
-     *
-     * @param metatopic
-     * @param topic
+     * @see #subscribeAsProducer(String...)
+     * @param metatopic If the metatopic is different to the Default, need to be provided here
      */
     public void subscribeAsProducer(String metatopic, String... topic){
         this.subscribeAsProducer();
@@ -95,7 +99,7 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     }
 
     /**
-     * Close connection
+     * Close connection and send the remaind elements
      */
     public void unsubscribeAsProducer(){
         if( this.sender_instance == null) return;
@@ -113,17 +117,16 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     }
 
     /**
-     *
-     * @param topic
+     * @see #subscribeAsConsumer()
+     * @param topic list of topic where the consumer it will be consuming
      */
     public void subscribeAsConsumer(String... topic){
         this.subscribeAsProducer("default", topic);
     }
 
     /**
-     *
-     * @param metatopic
-     * @param topic
+     * @see #subscribeAsProducer(String...)
+     * @param metatopic If the metatopic is different to the Default, need to be provided here
      */
     public void subscribeAsConsumer(String metatopic, String... topic){
         this.subscribeAsConsumer();
@@ -134,7 +137,7 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     }
 
     /**
-     * Close connection
+     * Close connection and stop consuming elements form the sidecar pipeline
      */
     public void unsubscribeAsConsumer() {
         if( this.receiver_instance == null) return;
@@ -142,7 +145,7 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     }
 
     /**
-     *
+     * Close the {@link Sender} and {@link Receiver}
      */
     public void close(){
         this.unsubscribeAsConsumer();
@@ -153,13 +156,14 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     public abstract boolean hasNext();
 
     @Override
-    public abstract T next();
+    public abstract T_IN next();
 
     /**
+     * Get the last elements received to be injected on the main pipeline.
      *
-     * @return
+     * @return {@link Iterator} with the last element on the {@link org.apache.wayang.plugin.hackit.core.sniffer.shipper.receiver.BufferReceiver}
      */
-    public Iterator<T> getNexts(){
+    public Iterator<T_IN> getNexts(){
         if( this.receiver_instance == null){
             throw new RuntimeException("The Receiver of the Shipper is not instanciated");
         }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java
index a13cabb..5e38a35 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java
@@ -22,27 +22,45 @@ import java.util.Queue;
 import java.util.concurrent.LinkedBlockingQueue;
 
 /**
+ * BufferReceiver is component where the element getted by the {@link Receiver} are stored waiting for be consumed
  *
- * @param <T>
+ * @param <T> type of the element that it will receive
  */
+//TODO: Implement this class well
 public class BufferReceiver<T> implements Serializable {
+
     //TODO implement the doble buffering
+    /**
+     * queue is a {@link Queue} where the element are storage waiting to be consumed
+     */
     private transient Queue<T> queue;
 
+    /**
+     * Start the Buffer to be enable to get new {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+     *
+     * @return True if the Buffer start without problem, False in other cases
+     */
     //TODO implement the server to receive the messages
     public boolean start(){
         return true;
     }
 
-    //TODO registrer on the rest of the worker
+
+    //TODO registrer on the rest of the worker; validate if is need
     public boolean register(){
         return true;
     }
 
+    //TODO Validate if is need
     public boolean existQueue(){
         return false;
     }
 
+    /**
+     * Insert a new value on the {@link BufferReceiver}
+     *
+     * @param value to be insert on the buffer
+     */
     public void put(T value){
         if(this.queue == null){
             this.queue = new LinkedBlockingQueue<>();
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java
index fdfe139..ece71cc 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java
@@ -22,8 +22,9 @@ import java.util.Collections;
 import java.util.Iterator;
 
 /**
+ * EmptyReceiver is a dummy implementation of {@link Receiver}
  *
- * @param <T>
+ * @param <T> type of process data
  */
 public class EmptyReceiver<T> extends Receiver<T> {
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java
index 80f9a9d..a529161 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java
@@ -21,29 +21,32 @@ import java.io.Serializable;
 import java.util.Iterator;
 
 /**
+ * Receiver is the component that handle the connection with the side car, and get
+ * external elements, this can be instructions to perform or new {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
  *
- * @param <T>
+ * @param <T> Type of received elements
  */
 public abstract class Receiver<T> implements Serializable {
 
     /**
-     *
+     * bufferReceiver is an instance of {@link BufferReceiver}
      */
-    private transient BufferReceiver<T> bufferReceive;
+    private transient BufferReceiver<T> bufferReceiver;
 
     /**
-     *
+     * Start the the Receiver service that will be waiting the new elements.
      */
     public abstract void init();
 
     /**
+     * Provide the newest elements received, either the process {@link #init()} or the previous call of {@link #getElements()}
      *
-     * @return
+     * @return {@link Iterator} with the elements
      */
     public abstract Iterator<T> getElements();
 
     /**
-     *
+     * Stop the service and clean the {@link BufferReceiver}
      */
     public abstract void close();
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java
index 1a2c459..d4a1e1f 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java
@@ -19,7 +19,7 @@
 package org.apache.wayang.plugin.hackit.core.sniffer.shipper.sender;
 
 /**
- *
+ * EmptySender is a dummy implementation of the {@link Sender}
  * @param <T>
  */
 public class EmptySender<T> implements Sender<T> {
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java
index 82a491c..2e6ad41 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java
@@ -20,24 +20,29 @@ package org.apache.wayang.plugin.hackit.core.sniffer.shipper.sender;
 import java.io.Serializable;
 
 /**
+ * Sender is the component that send the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple} out from
+ * the main pipeline
  *
- * @param <T>
+ * @param <T> type of the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple} that will be send out
  */
 public interface Sender<T> extends Serializable {
 
     /**
-     *
+     * Start the service or connect to the server where the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+     * need to be sended it
      */
     void init();
 
     /**
+     * Place in a buffer or send inmediatly the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple} this will
+     * depends on the configuration
      *
-     * @param value
+     * @param value to be send out
      */
     void send(T value);
 
     /**
-     *
+     * Terminate the connection and clean the buffers if is need
      */
     void close();
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java
index 16419db..6e49b3a 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java
@@ -24,7 +24,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 /**
- *
+ * TODO validate if is correct implementation
  */
 public class CollectionTagsToSniff implements Sniff {
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java
index 42da261..9fcf1b7 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java
@@ -21,7 +21,7 @@ package org.apache.wayang.plugin.hackit.core.sniffer.sniff;
 import org.apache.wayang.plugin.hackit.core.tags.HackitTag;
 
 /**
- *
+ * TODO validate if is correct implementation
  */
 public class SingleTagToSniff implements Sniff {
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java
index 3a57c6f..cd4461b 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java
@@ -20,15 +20,19 @@ package org.apache.wayang.plugin.hackit.core.sniffer.sniff;
 import java.io.Serializable;
 
 /**
+ * Sniff is the component that evaluate if some {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple} need to
+ * be treated on some way
  *
- * @param <I>
+ * @param <I> type of the element that will be evaluated
  */
 public interface Sniff<I> extends Serializable {
 
     /**
+     * sniff evaluate if the <code>input</code> need to be treated on some way or contains some {@link org.apache.wayang.plugin.hackit.core.tags.HackitTag}
+     * to enable the sniff
      *
-     * @param input
-     * @return
+     * @param input element to evaluate if is sniffable
+     * @return True if the <code>input</code> need to be sniffed, False in other cases
      */
     boolean sniff(I input);
 }

[incubator-wayang] 06/15: [WAYANG-31] split DataQuantaBuilder class on several class

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 5a4815c3ae7cecc0df880f8844613431027d1072
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Tue May 11 14:27:30 2021 -0400

    [WAYANG-31] split DataQuantaBuilder class on several class
---
 .../org/apache/wayang/api/DataQuantaBuilder.scala  | 1268 +-------------------
 .../org/apache/wayang/api/JavaPlanBuilder.scala    |    2 +-
 .../wayang/api/RecordDataQuantaBuilder.scala       |    1 +
 .../dataquantabuilder/BasicDataQuantaBuilder.scala |  152 +++
 .../CartesianDataQuantaBuilder.scala               |   43 +
 .../CoGroupDataQuantaBuilder.scala                 |  142 +++
 .../dataquantabuilder/CountDataQuantaBuilder.scala |   40 +
 .../CustomOperatorDataQuantaBuilder.scala          |   52 +
 .../DistinctDataQuantaBuilder.scala                |   39 +
 .../DoWhileDataQuantaBuilder.scala                 |  126 ++
 .../dataquantabuilder/FakeDataQuantaBuilder.scala  |   44 +
 .../FilterDataQuantaBuilder.scala                  |  102 ++
 .../FlatMapDataQuantaBuilder.scala                 |   91 ++
 .../GlobalGroupDataQuantaBuilder.scala             |   34 +
 .../GlobalReduceDataQuantaBuilder.scala            |   68 ++
 .../GroupByDataQuantaBuilder.scala                 |   76 ++
 .../IntersectDataQuantaBuilder.scala               |   39 +
 .../dataquantabuilder/JoinDataQuantaBuilder.scala  |  153 +++
 .../dataquantabuilder/KeyedDataQuantaBuilder.scala |   50 +
 .../LoadCollectionDataQuantaBuilder.scala          |   49 +
 .../dataquantabuilder/MapDataQuantaBuilder.scala   |   69 ++
 .../MapPartitionsDataQuantaBuilder.scala           |   93 ++
 .../ProjectionDataQuantaBuilder.scala              |   37 +
 .../ReduceByDataQuantaBuilder.scala                |   94 ++
 .../RepeatDataQuantaBuilder.scala                  |   51 +
 .../SampleDataQuantaBuilder.scala                  |   92 ++
 .../dataquantabuilder/SortDataQuantaBuilder.scala  |   97 ++
 .../UnarySourceDataQuantaBuilder.scala             |   37 +
 .../dataquantabuilder/UnionDataQuantaBuilder.scala |   39 +
 .../ZipWithIdDataQuantaBuilder.scala               |   41 +
 .../wayang/api/graph/EdgeDataQuantaBuilder.scala   |    3 +-
 .../java/org/apache/wayang/api/JavaApiTest.java    |    2 +
 .../java/org/apache/wayang/tests/RegressionIT.java |    4 +-
 33 files changed, 1960 insertions(+), 1270 deletions(-)

diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala
index a590795..c6e5dcb 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala
@@ -21,8 +21,8 @@ package org.apache.wayang.api
 
 import java.util.function.{Consumer, IntUnaryOperator, Function => JavaFunction}
 import java.util.{Collection => JavaCollection}
-
 import de.hpi.isg.profiledb.store.model.Experiment
+import org.apache.wayang.api.dataquantabuilder.{BasicDataQuantaBuilder, CartesianDataQuantaBuilder, CoGroupDataQuantaBuilder, CountDataQuantaBuilder, CustomOperatorDataQuantaBuilder, DistinctDataQuantaBuilder, DoWhileDataQuantaBuilder, FilterDataQuantaBuilder, FlatMapDataQuantaBuilder, GlobalGroupDataQuantaBuilder, GlobalReduceDataQuantaBuilder, GroupByDataQuantaBuilder, IntersectDataQuantaBuilder, JoinDataQuantaBuilder, KeyedDataQuantaBuilder, MapDataQuantaBuilder, MapPartitionsDataQuan [...]
 import org.apache.wayang.api.graph.{Edge, EdgeDataQuantaBuilder, EdgeDataQuantaBuilderDecorator}
 import org.apache.wayang.api.util.{DataQuantaBuilderCache, TypeTrap}
 import org.apache.wayang.basic.data.{Record, Tuple2 => RT2}
@@ -31,7 +31,7 @@ import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBiFunctio
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
 import org.apache.wayang.core.optimizer.cardinality.CardinalityEstimator
 import org.apache.wayang.core.optimizer.costs.{LoadEstimator, LoadProfile, LoadProfileEstimator}
-import org.apache.wayang.core.plan.wayangplan.{Operator, OutputSlot, WayangPlan, UnarySource}
+import org.apache.wayang.core.plan.wayangplan.{Operator, OutputSlot, UnarySource, WayangPlan}
 import org.apache.wayang.core.platform.Platform
 import org.apache.wayang.core.types.DataSetType
 import org.apache.wayang.core.util.{Logging, ReflectionUtils, WayangCollections, Tuple => WayangTuple}
@@ -438,1267 +438,3 @@ trait DataQuantaBuilder[+This <: DataQuantaBuilder[_, Out], Out] extends Logging
   protected[api] def dataQuanta(): DataQuanta[Out]
 
 }
-
-/**
-  * Abstract base class for builders of [[DataQuanta]]. The purpose of the builders is to provide a convenient
-  * Java API for Wayang that compensates for lacking default and named arguments.
-  */
-abstract class BasicDataQuantaBuilder[This <: DataQuantaBuilder[_, Out], Out](implicit _javaPlanBuilder: JavaPlanBuilder)
-  extends Logging with DataQuantaBuilder[This, Out] {
-
-  /**
-    * Lazy-initialized. The [[DataQuanta]] product of this builder.
-    */
-  private var result: DataQuanta[Out] = _
-
-  /**
-    * A name for the [[DataQuanta]] to be built.
-    */
-  private var name: String = _
-
-  /**
-    * An [[Experiment]] for the [[DataQuanta]] to be built.
-    */
-  private var experiment: Experiment = _
-
-  /**
-    * Broadcasts for the [[DataQuanta]] to be built.
-    */
-  private val broadcasts: ListBuffer[(String, DataQuantaBuilder[_, _])] = ListBuffer()
-
-  /**
-    * [[CardinalityEstimator]] for the [[DataQuanta]] to be built.
-    */
-  private var cardinalityEstimator: CardinalityEstimator = _
-
-  /**
-    * Target [[Platform]]s for the [[DataQuanta]] to be built.
-    */
-  private val targetPlatforms: ListBuffer[Platform] = ListBuffer()
-
-  /**
-    * Paths of UDF JAR files for the [[DataQuanta]] to be built.
-    */
-  private val udfJars: ListBuffer[String] = ListBuffer()
-
-  /**
-    * The type of the [[DataQuanta]] to be built.
-    */
-  protected[api] val outputTypeTrap = getOutputTypeTrap
-
-  /**
-    * Retrieve an intialization value for [[outputTypeTrap]].
-    *
-    * @return the [[TypeTrap]]
-    */
-  protected def getOutputTypeTrap = new TypeTrap
-
-  override protected[api] implicit def javaPlanBuilder = _javaPlanBuilder
-
-  override def withName(name: String): This = {
-    this.name = name
-    this.asInstanceOf[This]
-  }
-
-  override def withExperiment(experiment: Experiment): This = {
-    this.experiment = experiment
-    this.asInstanceOf[This]
-  }
-
-  override def withOutputType(outputType: DataSetType[Out]): This = {
-    this.outputTypeTrap.dataSetType = outputType
-    this.asInstanceOf[This]
-  }
-
-  override def withOutputClass(cls: Class[Out]): This = this.withOutputType(DataSetType.createDefault(cls))
-
-  override def withBroadcast[Sender <: DataQuantaBuilder[_, _]](sender: Sender, broadcastName: String): This = {
-    this.broadcasts += Tuple2(broadcastName, sender)
-    this.asInstanceOf[This]
-  }
-
-  override def withCardinalityEstimator(cardinalityEstimator: CardinalityEstimator): This = {
-    this.cardinalityEstimator = cardinalityEstimator
-    this.asInstanceOf[This]
-  }
-
-  override def withTargetPlatform(platform: Platform): This = {
-    this.targetPlatforms += platform
-    this.asInstanceOf[This]
-  }
-
-  def withUdfJarOf(cls: Class[_]): This = this.withUdfJar(ReflectionUtils.getDeclaringJar(cls))
-
-  override def withUdfJar(path: String): This = {
-    this.udfJars += path
-    this.asInstanceOf[This]
-  }
-
-  override protected[api] implicit def classTag: ClassTag[Out] = ClassTag(outputTypeTrap.typeClass)
-
-  override protected[api] def dataQuanta(): DataQuanta[Out] = {
-    if (this.result == null) {
-      this.result = this.build
-      if (this.name != null) this.result.withName(this.name)
-      if (this.cardinalityEstimator != null) this.result.withCardinalityEstimator(this.cardinalityEstimator)
-      if (this.experiment != null) this.result.withExperiment(experiment)
-      this.result.withUdfJars(this.udfJars: _*)
-      this.result.withTargetPlatforms(this.targetPlatforms: _*)
-      this.broadcasts.foreach {
-        case (broadcastName, senderBuilder) => this.result.withBroadcast(senderBuilder.dataQuanta(), broadcastName)
-      }
-    }
-    this.result
-  }
-
-  /**
-    * Create the [[DataQuanta]] built by this instance. Note the configuration being done in [[dataQuanta()]].
-    *
-    * @return the created and partially configured [[DataQuanta]]
-    */
-  protected def build: DataQuanta[Out]
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.core.plan.wayangplan.UnarySource]]s.
-  *
-  * @param source          the [[UnarySource]]
-  * @param javaPlanBuilder the [[JavaPlanBuilder]]
-  */
-class UnarySourceDataQuantaBuilder[This <: DataQuantaBuilder[_, Out], Out](source: UnarySource[Out])
-                                                                          (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[This, Out] {
-
-  override protected def build: DataQuanta[Out] = javaPlanBuilder.planBuilder.load(source)(this.classTag)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CollectionSource]]s.
-  *
-  * @param collection      the [[JavaCollection]] to be loaded
-  * @param javaPlanBuilder the [[JavaPlanBuilder]]
-  */
-class LoadCollectionDataQuantaBuilder[Out](collection: JavaCollection[Out])(implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[LoadCollectionDataQuantaBuilder[Out], Out] {
-
-  // Try to infer the type class from the collection.
-  locally {
-    if (!collection.isEmpty) {
-      val any = WayangCollections.getAny(collection)
-      if (any != null) {
-        this.outputTypeTrap.dataSetType = DataSetType.createDefault(any.getClass)
-      }
-    }
-  }
-
-  override protected def build: DataQuanta[Out] = javaPlanBuilder.planBuilder.loadCollection(collection)(this.classTag)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param udf             UDF for the [[MapOperator]]
-  */
-class MapDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In],
-                                    udf: SerializableFunction[In, Out])
-                                   (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[MapDataQuantaBuilder[In, Out], Out] {
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  // Try to infer the type classes from the udf.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In] => inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-    parameters.get("Output") match {
-      case cls: Class[Out] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-  }
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().mapJava(udf, this.udfLoadProfileEstimator)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s with
-  * [[org.apache.wayang.basic.function.ProjectionDescriptor]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param fieldNames      field names for the [[org.apache.wayang.basic.function.ProjectionDescriptor]]
-  */
-class ProjectionDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In], fieldNames: Array[String])
-                                          (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[ProjectionDataQuantaBuilder[In, Out], Out] {
-
-  override protected def build = inputDataQuanta.dataQuanta().project(fieldNames.toSeq)
-
-}
-
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param udf             UDF for the [[MapOperator]]
-  */
-class FilterDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T], udf: SerializablePredicate[T])
-                                (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[FilterDataQuantaBuilder[T], T] {
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  /** Selectivity of the filter predicate. */
-  private var selectivity: ProbabilisticDoubleInterval = _
-
-  /** SQL UDF implementing the filter predicate. */
-  private var sqlUdf: String = _
-
-  // Try to infer the type classes from the udf.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-  }
-
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  /**
-    * Add a SQL implementation of the UDF.
-    *
-    * @param sqlUdf a SQL condition that can be plugged into a `WHERE` clause
-    * @return this instance
-    */
-  def withSqlUdf(sqlUdf: String) = {
-    this.sqlUdf = sqlUdf
-    this
-  }
-
-  /**
-    * Specify the selectivity of the UDF.
-    *
-    * @param lowerEstimate the lower bound of the expected selectivity
-    * @param upperEstimate the upper bound of the expected selectivity
-    * @param confidence    the probability of the actual selectivity being within these bounds
-    * @return this instance
-    */
-  def withSelectivity(lowerEstimate: Double, upperEstimate: Double, confidence: Double) = {
-    this.selectivity = new ProbabilisticDoubleInterval(lowerEstimate, upperEstimate, confidence)
-    this
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().filterJava(
-    udf, this.sqlUdf, this.selectivity, this.udfLoadProfileEstimator
-  )
-
-}
-
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.SortOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param keyUdf             UDF for the [[org.apache.wayang.basic.operators.SortOperator]]
-  */
-class SortDataQuantaBuilder[T, Key](inputDataQuanta: DataQuantaBuilder[_, T],
-                                    keyUdf: SerializableFunction[T, Key])
-                                   (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[SortDataQuantaBuilder[T, Key], T] {
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  /** [[ClassTag]] or surrogate of [[Key]] */
-  implicit var keyTag: ClassTag[Key] = _
-
-  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf]]. */
-  private var keyUdfCpuEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf]]. */
-  private var keyUdfRamEstimator: LoadEstimator = _
-
-
-  // Try to infer the type classes from the UDFs.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(keyUdf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[T] => inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf)
-    }
-
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-
-
-  /**
-    * Set a [[LoadEstimator]] for the CPU load of the first key extraction UDF. Currently effectless.
-    *
-    * @param udfCpuEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThisKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
-    this.keyUdfCpuEstimator = udfCpuEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the RAM load of first the key extraction UDF. Currently effectless.
-    *
-    * @param udfRamEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThisKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
-    this.keyUdfRamEstimator = udfRamEstimator
-    this
-  }
-
-  override protected def build =
-    inputDataQuanta.dataQuanta().sortJava(keyUdf)(this.keyTag)
-
-}
-
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.FlatMapOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param udf             UDF for the [[org.apache.wayang.basic.operators.FlatMapOperator]]
-  */
-class FlatMapDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In],
-                                        udf: SerializableFunction[In, java.lang.Iterable[Out]])
-                                       (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[FlatMapDataQuantaBuilder[In, Out], Out] {
-
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  /** Selectivity of the filter predicate. */
-  private var selectivity: ProbabilisticDoubleInterval = _
-
-  // Try to infer the type classes from the udf.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In] => inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-    val originalClass = ReflectionUtils.getWrapperClass(parameters.get("Output"), 0)
-    originalClass match {
-      case cls: Class[Out] => {
-        this.outputTypeTrap.dataSetType= DataSetType.createDefault(cls)
-      }
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-  }
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  /**
-    * Specify the selectivity of the UDF.
-    *
-    * @param lowerEstimate the lower bound of the expected selectivity
-    * @param upperEstimate the upper bound of the expected selectivity
-    * @param confidence    the probability of the actual selectivity being within these bounds
-    * @return this instance
-    */
-  def withSelectivity(lowerEstimate: Double, upperEstimate: Double, confidence: Double) = {
-    this.selectivity = new ProbabilisticDoubleInterval(lowerEstimate, upperEstimate, confidence)
-    this
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().flatMapJava(
-    udf, this.selectivity, this.udfLoadProfileEstimator
-  )
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapPartitionsOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param udf             UDF for the [[org.apache.wayang.basic.operators.MapPartitionsOperator]]
-  */
-class MapPartitionsDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In],
-                                              udf: SerializableFunction[java.lang.Iterable[In], java.lang.Iterable[Out]])
-                                             (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[MapPartitionsDataQuantaBuilder[In, Out], Out] {
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  /** Selectivity of the filter predicate. */
-  private var selectivity: ProbabilisticDoubleInterval = _
-
-  // Try to infer the type classes from the udf.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In] => {
-        inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      }
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-    val originalClass = ReflectionUtils.getWrapperClass(parameters.get("Output"), 0)
-    originalClass match {
-      case cls: Class[Out] => {
-        this.outputTypeTrap.dataSetType= DataSetType.createDefault(cls)
-      }
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-  }
-
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  /**
-    * Specify the selectivity of the UDF.
-    *
-    * @param lowerEstimate the lower bound of the expected selectivity
-    * @param upperEstimate the upper bound of the expected selectivity
-    * @param confidence    the probability of the actual selectivity being within these bounds
-    * @return this instance
-    */
-  def withSelectivity(lowerEstimate: Double, upperEstimate: Double, confidence: Double) = {
-    this.selectivity = new ProbabilisticDoubleInterval(lowerEstimate, upperEstimate, confidence)
-    this
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().mapPartitionsJava(
-    udf, this.selectivity, this.udfLoadProfileEstimator
-  )
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.SampleOperator]]s.
-  *
-  * @param inputDataQuanta    [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param sampleSizeFunction the absolute size of the sample as a function of the current iteration number
-  */
-class SampleDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T], sampleSizeFunction: IntUnaryOperator)
-                                (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[SampleDataQuantaBuilder[T], T] {
-
-  /**
-    * Size of the dataset to be sampled.
-    */
-  private var datasetSize = SampleOperator.UNKNOWN_DATASET_SIZE
-
-  /**
-    * Sampling method to use.
-    */
-  private var sampleMethod = SampleOperator.Methods.ANY
-
-  /**
-    * Seed to use.
-    */
-  private var seed: Option[Long] = None
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  /**
-    * Set the size of the dataset that should be sampled.
-    *
-    * @param datasetSize the size of the dataset
-    * @return this instance
-    */
-  def withDatasetSize(datasetSize: Long) = {
-    this.datasetSize = datasetSize
-    this
-  }
-
-  /**
-    * Set the sample method to be used.
-    *
-    * @param sampleMethod the sample method
-    * @return this instance
-    */
-  def withSampleMethod(sampleMethod: SampleOperator.Methods) = {
-    this.sampleMethod = sampleMethod
-    this
-  }
-
-  /**
-    * Set the sample method to be used.
-    *
-    * @param seed
-    * @return this instance
-    */
-  def withSeed(seed: Long) = {
-    this.seed = Some(seed)
-    this
-  }
-
-  override protected def build =
-    inputDataQuanta.dataQuanta().sampleDynamicJava(sampleSizeFunction, this.datasetSize, this.seed, this.sampleMethod)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.ReduceByOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param udf             UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
-  * @param keyUdf          key extraction UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
-  */
-class ReduceByDataQuantaBuilder[Key, T](inputDataQuanta: DataQuantaBuilder[_, T],
-                                        keyUdf: SerializableFunction[T, Key],
-                                        udf: SerializableBinaryOperator[T])
-                                       (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[ReduceByDataQuantaBuilder[Key, T], T] {
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  implicit var keyTag: ClassTag[Key] = _
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  // TODO: Add these estimators.
-  //  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf]]. */
-  //  private var keyUdfCpuEstimator: LoadEstimator = _
-  //
-  //  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf]]. */
-  //  private var keyUdfRamEstimator: LoadEstimator = _
-
-  // Try to infer the type classes from the UDFs.
-  locally {
-    var parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableBinaryOperator[_]])
-    parameters.get("Type") match {
-      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-
-    parameters = ReflectionUtils.getTypeParameters(keyUdf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf)
-    }
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  override protected def build =
-    inputDataQuanta.dataQuanta().reduceByKeyJava(keyUdf, udf, this.udfLoadProfileEstimator)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param keyUdf          key extraction UDF for the [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]]
-  */
-class GroupByDataQuantaBuilder[Key, T](inputDataQuanta: DataQuantaBuilder[_, T], keyUdf: SerializableFunction[T, Key])
-                                      (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[GroupByDataQuantaBuilder[Key, T], java.lang.Iterable[T]] {
-
-  implicit var keyTag: ClassTag[Key] = _
-
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[keyUdf]]. */
-  private var keyUdfLoadProfileEstimator: LoadProfileEstimator = _
-
-  // Try to infer the type classes from the UDFs.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(keyUdf.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createGrouped(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf)
-    }
-
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withKeyUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.keyUdfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  override protected def build =
-    inputDataQuanta.dataQuanta().groupByKeyJava(keyUdf, this.keyUdfLoadProfileEstimator)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.GlobalMaterializedGroupOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  */
-class GlobalGroupDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])(implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[GlobalGroupDataQuantaBuilder[T], java.lang.Iterable[T]] {
-
-  override protected def build = inputDataQuanta.dataQuanta().group()
-
-}
-
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.GlobalReduceOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param udf             UDF for the [[org.apache.wayang.basic.operators.GlobalReduceOperator]]
-  */
-class GlobalReduceDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T],
-                                       udf: SerializableBinaryOperator[T])
-                                      (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[GlobalReduceDataQuantaBuilder[T], T] {
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  // Try to infer the type classes from the udf.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableBinaryOperator[_]])
-    parameters.get("Type") match {
-      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", udf)
-    }
-  }
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().reduceJava(udf, this.udfLoadProfileEstimator)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.UnionAllOperator]]s.
-  *
-  * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  */
-class UnionDataQuantaBuilder[T](inputDataQuanta0: DataQuantaBuilder[_, T],
-                                inputDataQuanta1: DataQuantaBuilder[_, T])
-                               (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[UnionDataQuantaBuilder[T], T] {
-
-  override def getOutputTypeTrap = inputDataQuanta0.outputTypeTrap
-
-  override protected def build = inputDataQuanta0.dataQuanta().union(inputDataQuanta1.dataQuanta())
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.IntersectOperator]]s.
-  *
-  * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  */
-class IntersectDataQuantaBuilder[T](inputDataQuanta0: DataQuantaBuilder[_, T],
-                                    inputDataQuanta1: DataQuantaBuilder[_, T])
-                                   (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[IntersectDataQuantaBuilder[T], T] {
-
-  override def getOutputTypeTrap = inputDataQuanta0.outputTypeTrap
-
-  override protected def build = inputDataQuanta0.dataQuanta().intersect(inputDataQuanta1.dataQuanta())
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.JoinOperator]]s.
-  *
-  * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param keyUdf0          first key extraction UDF for the [[org.apache.wayang.basic.operators.JoinOperator]]
-  * @param keyUdf1          first key extraction UDF for the [[org.apache.wayang.basic.operators.JoinOperator]]
-  */
-class JoinDataQuantaBuilder[In0, In1, Key](inputDataQuanta0: DataQuantaBuilder[_, In0],
-                                           inputDataQuanta1: DataQuantaBuilder[_, In1],
-                                           keyUdf0: SerializableFunction[In0, Key],
-                                           keyUdf1: SerializableFunction[In1, Key])
-                                          (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[JoinDataQuantaBuilder[In0, In1, Key], RT2[In0, In1]] {
-
-  /** [[ClassTag]] or surrogate of [[Key]] */
-  implicit var keyTag: ClassTag[Key] = _
-
-  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf0]]. */
-  private var keyUdf0CpuEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf0]]. */
-  private var keyUdf0RamEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf1]]. */
-  private var keyUdf1CpuEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf1]]. */
-  private var keyUdf1RamEstimator: LoadEstimator = _
-
-  // Try to infer the type classes from the UDFs.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(keyUdf0.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In0] => inputDataQuanta0.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
-    }
-
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf0)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(keyUdf1.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In1] => inputDataQuanta1.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
-    }
-
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf0)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-  // Since we are currently not looking at type parameters, we can statically determine the output type.
-  locally {
-    this.outputTypeTrap.dataSetType = dataSetType[RT2[_, _]]
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the CPU load of the first key extraction UDF. Currently effectless.
-    *
-    * @param udfCpuEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThisKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
-    this.keyUdf0CpuEstimator = udfCpuEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the RAM load of first the key extraction UDF. Currently effectless.
-    *
-    * @param udfRamEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThisKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
-    this.keyUdf0RamEstimator = udfRamEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the CPU load of the second key extraction UDF. Currently effectless.
-    *
-    * @param udfCpuEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThatKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
-    this.keyUdf1CpuEstimator = udfCpuEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the RAM load of the second key extraction UDF. Currently effectless.
-    *
-    * @param udfRamEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThatKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
-    this.keyUdf1RamEstimator = udfRamEstimator
-    this
-  }
-
-  /**
-    * Assemble the joined elements to new elements.
-    *
-    * @param udf produces a joined element from two joinable elements
-    * @return a new [[DataQuantaBuilder]] representing the assembled join product
-    */
-  def assemble[NewOut](udf: SerializableBiFunction[In0, In1, NewOut]) =
-    this.map(new SerializableFunction[RT2[In0, In1], NewOut] {
-      override def apply(joinTuple: RT2[In0, In1]): NewOut = udf.apply(joinTuple.field0, joinTuple.field1)
-    })
-
-  override protected def build =
-    inputDataQuanta0.dataQuanta().joinJava(keyUdf0, inputDataQuanta1.dataQuanta(), keyUdf1)(inputDataQuanta1.classTag, this.keyTag)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CoGroupOperator]]s.
-  *
-  * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param keyUdf0          first key extraction UDF for the [[org.apache.wayang.basic.operators.CoGroupOperator]]
-  * @param keyUdf1          first key extraction UDF for the [[org.apache.wayang.basic.operators.CoGroupOperator]]
-  */
-class CoGroupDataQuantaBuilder[In0, In1, Key](inputDataQuanta0: DataQuantaBuilder[_, In0],
-                                           inputDataQuanta1: DataQuantaBuilder[_, In1],
-                                           keyUdf0: SerializableFunction[In0, Key],
-                                           keyUdf1: SerializableFunction[In1, Key])
-                                          (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[CoGroupDataQuantaBuilder[In0, In1, Key], RT2[java.lang.Iterable[In0], java.lang.Iterable[In1]]] {
-
-  /** [[ClassTag]] or surrogate of [[Key]] */
-  implicit var keyTag: ClassTag[Key] = _
-
-  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf0]]. */
-  private var keyUdf0CpuEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf0]]. */
-  private var keyUdf0RamEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf1]]. */
-  private var keyUdf1CpuEstimator: LoadEstimator = _
-
-  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf1]]. */
-  private var keyUdf1RamEstimator: LoadEstimator = _
-
-  // Try to infer the type classes from the UDFs.
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(keyUdf0.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In0] => inputDataQuanta0.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
-    }
-
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf0)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-  locally {
-    val parameters = ReflectionUtils.getTypeParameters(keyUdf1.getClass, classOf[SerializableFunction[_, _]])
-    parameters.get("Input") match {
-      case cls: Class[In1] => inputDataQuanta1.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
-      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
-    }
-
-    this.keyTag = parameters.get("Output") match {
-      case cls: Class[Key] => ClassTag(cls)
-      case _ =>
-        logger.warn("Could not infer types from {}.", keyUdf0)
-        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-    }
-  }
-  // Since we are currently not looking at type parameters, we can statically determine the output type.
-  locally {
-    this.outputTypeTrap.dataSetType = dataSetType[RT2[_, _]]
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the CPU load of the first key extraction UDF. Currently effectless.
-    *
-    * @param udfCpuEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThisKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
-    this.keyUdf0CpuEstimator = udfCpuEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the RAM load of first the key extraction UDF. Currently effectless.
-    *
-    * @param udfRamEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThisKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
-    this.keyUdf0RamEstimator = udfRamEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the CPU load of the second key extraction UDF. Currently effectless.
-    *
-    * @param udfCpuEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThatKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
-    this.keyUdf1CpuEstimator = udfCpuEstimator
-    this
-  }
-
-  /**
-    * Set a [[LoadEstimator]] for the RAM load of the second key extraction UDF. Currently effectless.
-    *
-    * @param udfRamEstimator the [[LoadEstimator]]
-    * @return this instance
-    */
-  def withThatKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
-    this.keyUdf1RamEstimator = udfRamEstimator
-    this
-  }
-
-  override protected def build =
-    inputDataQuanta0.dataQuanta().coGroupJava(keyUdf0, inputDataQuanta1.dataQuanta(), keyUdf1)(inputDataQuanta1.classTag, this.keyTag)
-
-}
-
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CartesianOperator]]s.
-  *
-  * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
-  */
-class CartesianDataQuantaBuilder[In0, In1](inputDataQuanta0: DataQuantaBuilder[_, In0],
-                                           inputDataQuanta1: DataQuantaBuilder[_, In1])
-                                          (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[CartesianDataQuantaBuilder[In0, In1], RT2[In0, In1]] {
-
-  // Since we are currently not looking at type parameters, we can statically determine the output type.
-  locally {
-    this.outputTypeTrap.dataSetType = dataSetType[RT2[_, _]]
-  }
-
-  override protected def build =
-    inputDataQuanta0.dataQuanta().cartesian(inputDataQuanta1.dataQuanta())(inputDataQuanta1.classTag)
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.ZipWithIdOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  */
-class ZipWithIdDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])
-                                   (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[ZipWithIdDataQuantaBuilder[T], RT2[java.lang.Long, T]] {
-
-  // Since we are currently not looking at type parameters, we can statically determine the output type.
-  locally {
-    this.outputTypeTrap.dataSetType = dataSetType[RT2[_, _]]
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().zipWithId
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DistinctOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  */
-class DistinctDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])
-                                  (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[DistinctDataQuantaBuilder[T], T] {
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  override protected def build = inputDataQuanta.dataQuanta().distinct
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CountOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  */
-class CountDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])
-                               (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[CountDataQuantaBuilder[T], java.lang.Long] {
-
-  // We can statically determine the output type.
-  locally {
-    this.outputTypeTrap.dataSetType = dataSetType[java.lang.Long]
-  }
-
-  override protected def build = inputDataQuanta.dataQuanta().count
-
-}
-
-
-/**
-  * [[DataQuantaBuilder]] implementation for any [[org.apache.wayang.core.plan.wayangplan.Operator]]s. Does not offer
-  * any convenience methods, though.
-  *
-  * @param operator        the custom [[org.apache.wayang.core.plan.wayangplan.Operator]]
-  * @param outputIndex     index of the [[OutputSlot]] addressed by the new instance
-  * @param buildCache      a [[DataQuantaBuilderCache]] that must be shared across instances addressing the same [[Operator]]
-  * @param inputDataQuanta [[DataQuantaBuilder]]s for the input [[DataQuanta]]
-  * @param javaPlanBuilder the [[JavaPlanBuilder]] used to construct the current [[WayangPlan]]
-  */
-class CustomOperatorDataQuantaBuilder[T](operator: Operator,
-                                         outputIndex: Int,
-                                         buildCache: DataQuantaBuilderCache,
-                                         inputDataQuanta: DataQuantaBuilder[_, _]*)
-                                        (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[DataQuantaBuilder[_, T], T] {
-
-  override protected def build = {
-    // If the [[operator]] has multiple [[OutputSlot]]s, make sure that we only execute the build once.
-    if (!buildCache.hasCached) {
-      val dataQuanta = javaPlanBuilder.planBuilder.customOperator(operator, inputDataQuanta.map(_.dataQuanta()): _*)
-      buildCache.cache(dataQuanta)
-    }
-    buildCache(outputIndex)
-  }
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DoWhileOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param conditionUdf    UDF for the looping condition
-  * @param bodyBuilder     builds the loop body
-  */
-class DoWhileDataQuantaBuilder[T, ConvOut](inputDataQuanta: DataQuantaBuilder[_, T],
-                                           conditionUdf: SerializablePredicate[JavaCollection[ConvOut]],
-                                           bodyBuilder: JavaFunction[DataQuantaBuilder[_, T], WayangTuple[DataQuantaBuilder[_, T], DataQuantaBuilder[_, ConvOut]]])
-                                          (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[DoWhileDataQuantaBuilder[T, ConvOut], T] {
-
-  // TODO: Get the ClassTag right.
-  implicit private var convOutClassTag: ClassTag[ConvOut] = ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  // TODO: We could improve by combining the TypeTraps in the body loop.
-
-  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the UDF. */
-  private var udfLoadProfileEstimator: LoadProfileEstimator = _
-
-  /** Number of expected iterations. */
-  private var numExpectedIterations = 20
-
-  /**
-    * Set a [[LoadProfileEstimator]] for the load of the UDF.
-    *
-    * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
-    * @return this instance
-    */
-  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
-    this.udfLoadProfileEstimator = udfLoadProfileEstimator
-    this
-  }
-
-  /**
-    * Explicitly set the [[DataSetType]] for the condition [[DataQuanta]]. Note that it is not
-    * always necessary to set it and that it can be inferred in some situations.
-    *
-    * @param outputType the output [[DataSetType]]
-    * @return this instance
-    */
-  def withConditionType(outputType: DataSetType[ConvOut]) = {
-    this.convOutClassTag = ClassTag(outputType.getDataUnitType.getTypeClass)
-    this
-  }
-
-  /**
-    * Explicitly set the [[Class]] for the condition [[DataQuanta]]. Note that it is not
-    * always necessary to set it and that it can be inferred in some situations.
-    *
-    * @param cls the output [[Class]]
-    * @return this instance
-    */
-  def withConditionClass(cls: Class[ConvOut]) = {
-    this.convOutClassTag = ClassTag(cls)
-    this
-  }
-
-  /**
-    * Set the number of expected iterations for the built [[org.apache.wayang.basic.operators.DoWhileOperator]].
-    *
-    * @param numExpectedIterations the expected number of iterations
-    * @return this instance
-    */
-  def withExpectedNumberOfIterations(numExpectedIterations: Int) = {
-    this.numExpectedIterations = numExpectedIterations
-    this
-  }
-
-  override protected def build =
-    inputDataQuanta.dataQuanta().doWhileJava[ConvOut](
-      conditionUdf, dataQuantaBodyBuilder, this.numExpectedIterations, this.udfLoadProfileEstimator
-    )(this.convOutClassTag)
-
-
-  /**
-    * Create a loop body builder that is based on [[DataQuanta]].
-    *
-    * @return the loop body builder
-    */
-  private def dataQuantaBodyBuilder =
-    new JavaFunction[DataQuanta[T], WayangTuple[DataQuanta[T], DataQuanta[ConvOut]]] {
-      override def apply(loopStart: DataQuanta[T]) = {
-        val loopStartBuilder = new FakeDataQuantaBuilder(loopStart)
-        val loopEndBuilders = bodyBuilder(loopStartBuilder)
-        new WayangTuple(loopEndBuilders.field0.dataQuanta(), loopEndBuilders.field1.dataQuanta())
-      }
-    }
-
-}
-
-/**
-  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DoWhileOperator]]s.
-  *
-  * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
-  * @param numRepetitions  number of repetitions of the loop
-  * @param bodyBuilder     builds the loop body
-  */
-class RepeatDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T],
-                                 numRepetitions: Int,
-                                 bodyBuilder: JavaFunction[DataQuantaBuilder[_, T], DataQuantaBuilder[_, T]])
-                                (implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[RepeatDataQuantaBuilder[T], T] {
-
-  // Reuse the input TypeTrap to enforce type equality between input and output.
-  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
-
-  // TODO: We could improve by combining the TypeTraps in the body loop.
-
-  override protected def build =
-    inputDataQuanta.dataQuanta().repeat(numRepetitions, startDataQuanta => {
-      val loopStartbuilder = new FakeDataQuantaBuilder(startDataQuanta)
-      bodyBuilder(loopStartbuilder).dataQuanta()
-    })
-
-}
-
-/**
-  * Wraps [[DataQuanta]] and exposes them as [[DataQuantaBuilder]], i.e., this is an adapter.
-  *
-  * @param _dataQuanta the wrapped [[DataQuanta]]
-  */
-class FakeDataQuantaBuilder[T](_dataQuanta: DataQuanta[T])(implicit javaPlanBuilder: JavaPlanBuilder)
-  extends BasicDataQuantaBuilder[FakeDataQuantaBuilder[T], T] {
-
-  override implicit def classTag = ClassTag(_dataQuanta.output.getType.getDataUnitType.getTypeClass)
-
-  override def dataQuanta() = _dataQuanta
-
-  /**
-    * Create the [[DataQuanta]] built by this instance. Note the configuration being done in [[dataQuanta()]].
-    *
-    * @return the created and partially configured [[DataQuanta]]
-    */
-  override protected def build: DataQuanta[T] = _dataQuanta
-}
-
-/**
-  * This is not an actual [[DataQuantaBuilder]] but rather decorates such a [[DataQuantaBuilder]] with a key.
-  */
-class KeyedDataQuantaBuilder[Out, Key](private val dataQuantaBuilder: DataQuantaBuilder[_, Out],
-                                       private val keyExtractor: SerializableFunction[Out, Key])
-                                      (implicit javaPlanBuilder: JavaPlanBuilder) {
-
-  /**
-    * Joins this instance with the given one via their keys.
-    *
-    * @param that the instance to join with
-    * @return a [[DataQuantaBuilder]] representing the join product
-    */
-  def join[ThatOut](that: KeyedDataQuantaBuilder[ThatOut, Key]) =
-    dataQuantaBuilder.join(this.keyExtractor, that.dataQuantaBuilder, that.keyExtractor)
-
-  /**
-    * Co-groups this instance with the given one via their keys.
-    *
-    * @param that the instance to join with
-    * @return a [[DataQuantaBuilder]] representing the co-group product
-    */
-  def coGroup[ThatOut](that: KeyedDataQuantaBuilder[ThatOut, Key]) =
-    dataQuantaBuilder.coGroup(this.keyExtractor, that.dataQuantaBuilder, that.keyExtractor)
-
-}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala
index dc9f6be..2c88f58 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala
@@ -19,9 +19,9 @@
 package org.apache.wayang.api
 
 import java.util.{Collection => JavaCollection}
-
 import de.hpi.isg.profiledb.store.model.Experiment
 import org.apache.commons.lang3.Validate
+import org.apache.wayang.api.dataquantabuilder.{CustomOperatorDataQuantaBuilder, LoadCollectionDataQuantaBuilder, UnarySourceDataQuantaBuilder}
 import org.apache.wayang.api.util.DataQuantaBuilderCache
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.operators.{TableSource, TextFileSource}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala
index 5af271f..5dc045b 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala
@@ -18,6 +18,7 @@
 
 package org.apache.wayang.api
 
+import org.apache.wayang.api.dataquantabuilder.BasicDataQuantaBuilder
 import org.apache.wayang.api.util.DataQuantaBuilderDecorator
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.function.ProjectionDescriptor
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/BasicDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/BasicDataQuantaBuilder.scala
new file mode 100644
index 0000000..9aad1a1
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/BasicDataQuantaBuilder.scala
@@ -0,0 +1,152 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import de.hpi.isg.profiledb.store.model.Experiment
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.optimizer.cardinality.CardinalityEstimator
+import org.apache.wayang.core.platform.Platform
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.{Logging, ReflectionUtils}
+
+import scala.collection.mutable.ListBuffer
+import scala.reflect.ClassTag
+
+/**
+ * Abstract base class for builders of [[DataQuanta]]. The purpose of the builders is to provide a convenient
+ * Java API for Wayang that compensates for lacking default and named arguments.
+ */
+abstract class BasicDataQuantaBuilder[This <: DataQuantaBuilder[_, Out], Out](implicit _javaPlanBuilder: JavaPlanBuilder)
+  extends Logging with DataQuantaBuilder[This, Out] {
+
+  /**
+   * Lazy-initialized. The [[DataQuanta]] product of this builder.
+   */
+  private var result: DataQuanta[Out] = _
+
+  /**
+   * A name for the [[DataQuanta]] to be built.
+   */
+  private var name: String = _
+
+  /**
+   * An [[Experiment]] for the [[DataQuanta]] to be built.
+   */
+  private var experiment: Experiment = _
+
+  /**
+   * Broadcasts for the [[DataQuanta]] to be built.
+   */
+  private val broadcasts: ListBuffer[(String, DataQuantaBuilder[_, _])] = ListBuffer()
+
+  /**
+   * [[CardinalityEstimator]] for the [[DataQuanta]] to be built.
+   */
+  private var cardinalityEstimator: CardinalityEstimator = _
+
+  /**
+   * Target [[Platform]]s for the [[DataQuanta]] to be built.
+   */
+  private val targetPlatforms: ListBuffer[Platform] = ListBuffer()
+
+  /**
+   * Paths of UDF JAR files for the [[DataQuanta]] to be built.
+   */
+  private val udfJars: ListBuffer[String] = ListBuffer()
+
+  /**
+   * The type of the [[DataQuanta]] to be built.
+   */
+  protected[api] val outputTypeTrap = getOutputTypeTrap
+
+  /**
+   * Retrieve an intialization value for [[outputTypeTrap]].
+   *
+   * @return the [[TypeTrap]]
+   */
+  protected def getOutputTypeTrap = new TypeTrap
+
+  override protected[api] implicit def javaPlanBuilder = _javaPlanBuilder
+
+  override def withName(name: String): This = {
+    this.name = name
+    this.asInstanceOf[This]
+  }
+
+  override def withExperiment(experiment: Experiment): This = {
+    this.experiment = experiment
+    this.asInstanceOf[This]
+  }
+
+  override def withOutputType(outputType: DataSetType[Out]): This = {
+    this.outputTypeTrap.dataSetType = outputType
+    this.asInstanceOf[This]
+  }
+
+  override def withOutputClass(cls: Class[Out]): This = this.withOutputType(DataSetType.createDefault(cls))
+
+  override def withBroadcast[Sender <: DataQuantaBuilder[_, _]](sender: Sender, broadcastName: String): This = {
+    this.broadcasts += Tuple2(broadcastName, sender)
+    this.asInstanceOf[This]
+  }
+
+  override def withCardinalityEstimator(cardinalityEstimator: CardinalityEstimator): This = {
+    this.cardinalityEstimator = cardinalityEstimator
+    this.asInstanceOf[This]
+  }
+
+  override def withTargetPlatform(platform: Platform): This = {
+    this.targetPlatforms += platform
+    this.asInstanceOf[This]
+  }
+
+  def withUdfJarOf(cls: Class[_]): This = this.withUdfJar(ReflectionUtils.getDeclaringJar(cls))
+
+  override def withUdfJar(path: String): This = {
+    this.udfJars += path
+    this.asInstanceOf[This]
+  }
+
+  override protected[api] implicit def classTag: ClassTag[Out] = ClassTag(outputTypeTrap.typeClass)
+
+  override protected[api] def dataQuanta(): DataQuanta[Out] = {
+    if (this.result == null) {
+      this.result = this.build
+      if (this.name != null) this.result.withName(this.name)
+      if (this.cardinalityEstimator != null) this.result.withCardinalityEstimator(this.cardinalityEstimator)
+      if (this.experiment != null) this.result.withExperiment(experiment)
+      this.result.withUdfJars(this.udfJars: _*)
+      this.result.withTargetPlatforms(this.targetPlatforms: _*)
+      this.broadcasts.foreach {
+        case (broadcastName, senderBuilder) => this.result.withBroadcast(senderBuilder.dataQuanta(), broadcastName)
+      }
+    }
+    this.result
+  }
+
+  /**
+   * Create the [[DataQuanta]] built by this instance. Note the configuration being done in [[dataQuanta()]].
+   *
+   * @return the created and partially configured [[DataQuanta]]
+   */
+  protected def build: DataQuanta[Out]
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CartesianDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CartesianDataQuantaBuilder.scala
new file mode 100644
index 0000000..40d4fae
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CartesianDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.basic.data.{Tuple2 => WT2}
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CartesianOperator]]s.
+ *
+ * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ */
+class CartesianDataQuantaBuilder[In0, In1](inputDataQuanta0: DataQuantaBuilder[_, In0],
+                                           inputDataQuanta1: DataQuantaBuilder[_, In1])
+                                          (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[CartesianDataQuantaBuilder[In0, In1], WT2[In0, In1]] {
+
+  // Since we are currently not looking at type parameters, we can statically determine the output type.
+  locally {
+    this.outputTypeTrap.dataSetType = dataSetType[WT2[_, _]]
+  }
+
+  override protected def build =
+    inputDataQuanta0.dataQuanta().cartesian(inputDataQuanta1.dataQuanta())(inputDataQuanta1.classTag)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CoGroupDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CoGroupDataQuantaBuilder.scala
new file mode 100644
index 0000000..aa1d88a
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CoGroupDataQuantaBuilder.scala
@@ -0,0 +1,142 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.basic.data.{Tuple2 => WT2}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.costs.LoadEstimator
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+import scala.reflect.ClassTag
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CoGroupOperator]]s.
+ *
+ * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param keyUdf0          first key extraction UDF for the [[org.apache.wayang.basic.operators.CoGroupOperator]]
+ * @param keyUdf1          first key extraction UDF for the [[org.apache.wayang.basic.operators.CoGroupOperator]]
+ */
+class CoGroupDataQuantaBuilder[In0, In1, Key](inputDataQuanta0: DataQuantaBuilder[_, In0],
+                                              inputDataQuanta1: DataQuantaBuilder[_, In1],
+                                              keyUdf0: SerializableFunction[In0, Key],
+                                              keyUdf1: SerializableFunction[In1, Key])
+                                             (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[CoGroupDataQuantaBuilder[In0, In1, Key], WT2[java.lang.Iterable[In0], java.lang.Iterable[In1]]] {
+
+  /** [[ClassTag]] or surrogate of [[Key]] */
+  implicit var keyTag: ClassTag[Key] = _
+
+  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf0]]. */
+  private var keyUdf0CpuEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf0]]. */
+  private var keyUdf0RamEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf1]]. */
+  private var keyUdf1CpuEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf1]]. */
+  private var keyUdf1RamEstimator: LoadEstimator = _
+
+  // Try to infer the type classes from the UDFs.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(keyUdf0.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In0] => inputDataQuanta0.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
+    }
+
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf0)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(keyUdf1.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In1] => inputDataQuanta1.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
+    }
+
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf0)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+  // Since we are currently not looking at type parameters, we can statically determine the output type.
+  locally {
+    this.outputTypeTrap.dataSetType = dataSetType[WT2[_, _]]
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the CPU load of the first key extraction UDF. Currently effectless.
+   *
+   * @param udfCpuEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThisKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
+    this.keyUdf0CpuEstimator = udfCpuEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the RAM load of first the key extraction UDF. Currently effectless.
+   *
+   * @param udfRamEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThisKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
+    this.keyUdf0RamEstimator = udfRamEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the CPU load of the second key extraction UDF. Currently effectless.
+   *
+   * @param udfCpuEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThatKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
+    this.keyUdf1CpuEstimator = udfCpuEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the RAM load of the second key extraction UDF. Currently effectless.
+   *
+   * @param udfRamEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThatKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
+    this.keyUdf1RamEstimator = udfRamEstimator
+    this
+  }
+
+  override protected def build =
+    inputDataQuanta0.dataQuanta().coGroupJava(keyUdf0, inputDataQuanta1.dataQuanta(), keyUdf1)(inputDataQuanta1.classTag, this.keyTag)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CountDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CountDataQuantaBuilder.scala
new file mode 100644
index 0000000..d8f7d18
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CountDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CountOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ */
+class CountDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])
+                               (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[CountDataQuantaBuilder[T], java.lang.Long] {
+
+  // We can statically determine the output type.
+  locally {
+    this.outputTypeTrap.dataSetType = dataSetType[java.lang.Long]
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().count
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CustomOperatorDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CustomOperatorDataQuantaBuilder.scala
new file mode 100644
index 0000000..5fca840
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CustomOperatorDataQuantaBuilder.scala
@@ -0,0 +1,52 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.DataQuantaBuilderCache
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.plan.wayangplan.{Operator, OutputSlot, WayangPlan}
+
+/**
+ * [[DataQuantaBuilder]] implementation for any [[org.apache.wayang.core.plan.wayangplan.Operator]]s. Does not offer
+ * any convenience methods, though.
+ *
+ * @param operator        the custom [[org.apache.wayang.core.plan.wayangplan.Operator]]
+ * @param outputIndex     index of the [[OutputSlot]] addressed by the new instance
+ * @param buildCache      a [[DataQuantaBuilderCache]] that must be shared across instances addressing the same [[Operator]]
+ * @param inputDataQuanta [[DataQuantaBuilder]]s for the input [[DataQuanta]]
+ * @param javaPlanBuilder the [[JavaPlanBuilder]] used to construct the current [[WayangPlan]]
+ */
+class CustomOperatorDataQuantaBuilder[T](operator: Operator,
+                                         outputIndex: Int,
+                                         buildCache: DataQuantaBuilderCache,
+                                         inputDataQuanta: DataQuantaBuilder[_, _]*)
+                                        (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[DataQuantaBuilder[_, T], T] {
+
+  override protected def build = {
+    // If the [[operator]] has multiple [[OutputSlot]]s, make sure that we only execute the build once.
+    if (!buildCache.hasCached) {
+      val dataQuanta = javaPlanBuilder.planBuilder.customOperator(operator, inputDataQuanta.map(_.dataQuanta()): _*)
+      buildCache.cache(dataQuanta)
+    }
+    buildCache(outputIndex)
+  }
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DistinctDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DistinctDataQuantaBuilder.scala
new file mode 100644
index 0000000..9e8e044
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DistinctDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DistinctOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ */
+class DistinctDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])
+                                  (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[DistinctDataQuantaBuilder[T], T] {
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  override protected def build = inputDataQuanta.dataQuanta().distinct
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DoWhileDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DoWhileDataQuantaBuilder.scala
new file mode 100644
index 0000000..34de7be
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DoWhileDataQuantaBuilder.scala
@@ -0,0 +1,126 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializablePredicate
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.{Tuple => WayangTuple}
+
+import java.util.function.{Function => JavaFunction}
+import java.util.{Collection => JavaCollection}
+import scala.reflect.ClassTag
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DoWhileOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param conditionUdf    UDF for the looping condition
+ * @param bodyBuilder     builds the loop body
+ */
+class DoWhileDataQuantaBuilder[T, ConvOut](inputDataQuanta: DataQuantaBuilder[_, T],
+                                           conditionUdf: SerializablePredicate[JavaCollection[ConvOut]],
+                                           bodyBuilder: JavaFunction[DataQuantaBuilder[_, T], WayangTuple[DataQuantaBuilder[_, T], DataQuantaBuilder[_, ConvOut]]])
+                                          (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[DoWhileDataQuantaBuilder[T, ConvOut], T] {
+
+  // TODO: Get the ClassTag right.
+  implicit private var convOutClassTag: ClassTag[ConvOut] = ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  // TODO: We could improve by combining the TypeTraps in the body loop.
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the UDF. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  /** Number of expected iterations. */
+  private var numExpectedIterations = 20
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  /**
+   * Explicitly set the [[DataSetType]] for the condition [[DataQuanta]]. Note that it is not
+   * always necessary to set it and that it can be inferred in some situations.
+   *
+   * @param outputType the output [[DataSetType]]
+   * @return this instance
+   */
+  def withConditionType(outputType: DataSetType[ConvOut]) = {
+    this.convOutClassTag = ClassTag(outputType.getDataUnitType.getTypeClass)
+    this
+  }
+
+  /**
+   * Explicitly set the [[Class]] for the condition [[DataQuanta]]. Note that it is not
+   * always necessary to set it and that it can be inferred in some situations.
+   *
+   * @param cls the output [[Class]]
+   * @return this instance
+   */
+  def withConditionClass(cls: Class[ConvOut]) = {
+    this.convOutClassTag = ClassTag(cls)
+    this
+  }
+
+  /**
+   * Set the number of expected iterations for the built [[org.apache.wayang.basic.operators.DoWhileOperator]].
+   *
+   * @param numExpectedIterations the expected number of iterations
+   * @return this instance
+   */
+  def withExpectedNumberOfIterations(numExpectedIterations: Int) = {
+    this.numExpectedIterations = numExpectedIterations
+    this
+  }
+
+  override protected def build =
+    inputDataQuanta.dataQuanta().doWhileJava[ConvOut](
+      conditionUdf, dataQuantaBodyBuilder, this.numExpectedIterations, this.udfLoadProfileEstimator
+    )(this.convOutClassTag)
+
+
+  /**
+   * Create a loop body builder that is based on [[DataQuanta]].
+   *
+   * @return the loop body builder
+   */
+  private def dataQuantaBodyBuilder =
+    new JavaFunction[DataQuanta[T], WayangTuple[DataQuanta[T], DataQuanta[ConvOut]]] {
+      override def apply(loopStart: DataQuanta[T]) = {
+        val loopStartBuilder = new FakeDataQuantaBuilder(loopStart)
+        val loopEndBuilders = bodyBuilder(loopStartBuilder)
+        new WayangTuple(loopEndBuilders.field0.dataQuanta(), loopEndBuilders.field1.dataQuanta())
+      }
+    }
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FakeDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FakeDataQuantaBuilder.scala
new file mode 100644
index 0000000..6b52c40
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FakeDataQuantaBuilder.scala
@@ -0,0 +1,44 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+import scala.reflect.ClassTag
+
+/**
+ * Wraps [[DataQuanta]] and exposes them as [[DataQuantaBuilder]], i.e., this is an adapter.
+ *
+ * @param _dataQuanta the wrapped [[DataQuanta]]
+ */
+class FakeDataQuantaBuilder[T](_dataQuanta: DataQuanta[T])(implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[FakeDataQuantaBuilder[T], T] {
+
+  override implicit def classTag = ClassTag(_dataQuanta.output.getType.getDataUnitType.getTypeClass)
+
+  override def dataQuanta() = _dataQuanta
+
+  /**
+   * Create the [[DataQuanta]] built by this instance. Note the configuration being done in [[dataQuanta()]].
+   *
+   * @return the created and partially configured [[DataQuanta]]
+   */
+  override protected def build: DataQuanta[T] = _dataQuanta
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FilterDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FilterDataQuantaBuilder.scala
new file mode 100644
index 0000000..3d33c80
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FilterDataQuantaBuilder.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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.basic.operators.MapOperator
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableFunction, SerializablePredicate}
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param udf             UDF for the [[MapOperator]]
+ */
+class FilterDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T], udf: SerializablePredicate[T])
+                                (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[FilterDataQuantaBuilder[T], T] {
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  /** Selectivity of the filter predicate. */
+  private var selectivity: ProbabilisticDoubleInterval = _
+
+  /** SQL UDF implementing the filter predicate. */
+  private var sqlUdf: String = _
+
+  // Try to infer the type classes from the udf.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+  }
+
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  /**
+   * Add a SQL implementation of the UDF.
+   *
+   * @param sqlUdf a SQL condition that can be plugged into a `WHERE` clause
+   * @return this instance
+   */
+  def withSqlUdf(sqlUdf: String) = {
+    this.sqlUdf = sqlUdf
+    this
+  }
+
+  /**
+   * Specify the selectivity of the UDF.
+   *
+   * @param lowerEstimate the lower bound of the expected selectivity
+   * @param upperEstimate the upper bound of the expected selectivity
+   * @param confidence    the probability of the actual selectivity being within these bounds
+   * @return this instance
+   */
+  def withSelectivity(lowerEstimate: Double, upperEstimate: Double, confidence: Double) = {
+    this.selectivity = new ProbabilisticDoubleInterval(lowerEstimate, upperEstimate, confidence)
+    this
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().filterJava(
+    udf, this.sqlUdf, this.selectivity, this.udfLoadProfileEstimator
+  )
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FlatMapDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FlatMapDataQuantaBuilder.scala
new file mode 100644
index 0000000..83b4383
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FlatMapDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.FlatMapOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param udf             UDF for the [[org.apache.wayang.basic.operators.FlatMapOperator]]
+ */
+class FlatMapDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In],
+                                        udf: SerializableFunction[In, java.lang.Iterable[Out]])
+                                       (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[FlatMapDataQuantaBuilder[In, Out], Out] {
+
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  /** Selectivity of the filter predicate. */
+  private var selectivity: ProbabilisticDoubleInterval = _
+
+  // Try to infer the type classes from the udf.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In] => inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+    val originalClass = ReflectionUtils.getWrapperClass(parameters.get("Output"), 0)
+    originalClass match {
+      case cls: Class[Out] => {
+        this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      }
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+  }
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  /**
+   * Specify the selectivity of the UDF.
+   *
+   * @param lowerEstimate the lower bound of the expected selectivity
+   * @param upperEstimate the upper bound of the expected selectivity
+   * @param confidence    the probability of the actual selectivity being within these bounds
+   * @return this instance
+   */
+  def withSelectivity(lowerEstimate: Double, upperEstimate: Double, confidence: Double) = {
+    this.selectivity = new ProbabilisticDoubleInterval(lowerEstimate, upperEstimate, confidence)
+    this
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().flatMapJava(
+    udf, this.selectivity, this.udfLoadProfileEstimator
+  )
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalGroupDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalGroupDataQuantaBuilder.scala
new file mode 100644
index 0000000..abc1d9a
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalGroupDataQuantaBuilder.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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.GlobalMaterializedGroupOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ */
+class GlobalGroupDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])(implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[GlobalGroupDataQuantaBuilder[T], java.lang.Iterable[T]] {
+
+  override protected def build = inputDataQuanta.dataQuanta().group()
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalReduceDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalReduceDataQuantaBuilder.scala
new file mode 100644
index 0000000..c3efc95
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalReduceDataQuantaBuilder.scala
@@ -0,0 +1,68 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableBinaryOperator
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.GlobalReduceOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param udf             UDF for the [[org.apache.wayang.basic.operators.GlobalReduceOperator]]
+ */
+class GlobalReduceDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T],
+                                       udf: SerializableBinaryOperator[T])
+                                      (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[GlobalReduceDataQuantaBuilder[T], T] {
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  // Try to infer the type classes from the udf.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableBinaryOperator[_]])
+    parameters.get("Type") match {
+      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+  }
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().reduceJava(udf, this.udfLoadProfileEstimator)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GroupByDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GroupByDataQuantaBuilder.scala
new file mode 100644
index 0000000..b61c71e
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GroupByDataQuantaBuilder.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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+import scala.reflect.ClassTag
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param keyUdf          key extraction UDF for the [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]]
+ */
+class GroupByDataQuantaBuilder[Key, T](inputDataQuanta: DataQuantaBuilder[_, T], keyUdf: SerializableFunction[T, Key])
+                                      (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[GroupByDataQuantaBuilder[Key, T], java.lang.Iterable[T]] {
+
+  implicit var keyTag: ClassTag[Key] = _
+
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[keyUdf]]. */
+  private var keyUdfLoadProfileEstimator: LoadProfileEstimator = _
+
+  // Try to infer the type classes from the UDFs.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(keyUdf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createGrouped(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf)
+    }
+
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withKeyUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.keyUdfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  override protected def build =
+    inputDataQuanta.dataQuanta().groupByKeyJava(keyUdf, this.keyUdfLoadProfileEstimator)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/IntersectDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/IntersectDataQuantaBuilder.scala
new file mode 100644
index 0000000..6db8ffe
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/IntersectDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.IntersectOperator]]s.
+ *
+ * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ */
+class IntersectDataQuantaBuilder[T](inputDataQuanta0: DataQuantaBuilder[_, T],
+                                    inputDataQuanta1: DataQuantaBuilder[_, T])
+                                   (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[IntersectDataQuantaBuilder[T], T] {
+
+  override def getOutputTypeTrap = inputDataQuanta0.outputTypeTrap
+
+  override protected def build = inputDataQuanta0.dataQuanta().intersect(inputDataQuanta1.dataQuanta())
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/JoinDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/JoinDataQuantaBuilder.scala
new file mode 100644
index 0000000..ab49ffb
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/JoinDataQuantaBuilder.scala
@@ -0,0 +1,153 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.basic.data.{Tuple2 => WT2}
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBiFunction, SerializableFunction}
+import org.apache.wayang.core.optimizer.costs.LoadEstimator
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+import scala.reflect.ClassTag
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.JoinOperator]]s.
+ *
+ * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param keyUdf0          first key extraction UDF for the [[org.apache.wayang.basic.operators.JoinOperator]]
+ * @param keyUdf1          first key extraction UDF for the [[org.apache.wayang.basic.operators.JoinOperator]]
+ */
+class JoinDataQuantaBuilder[In0, In1, Key](inputDataQuanta0: DataQuantaBuilder[_, In0],
+                                           inputDataQuanta1: DataQuantaBuilder[_, In1],
+                                           keyUdf0: SerializableFunction[In0, Key],
+                                           keyUdf1: SerializableFunction[In1, Key])
+                                          (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[JoinDataQuantaBuilder[In0, In1, Key], WT2[In0, In1]] {
+
+  /** [[ClassTag]] or surrogate of [[Key]] */
+  implicit var keyTag: ClassTag[Key] = _
+
+  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf0]]. */
+  private var keyUdf0CpuEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf0]]. */
+  private var keyUdf0RamEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf1]]. */
+  private var keyUdf1CpuEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf1]]. */
+  private var keyUdf1RamEstimator: LoadEstimator = _
+
+  // Try to infer the type classes from the UDFs.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(keyUdf0.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In0] => inputDataQuanta0.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
+    }
+
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf0)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(keyUdf1.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In1] => inputDataQuanta1.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf0)
+    }
+
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf0)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+  // Since we are currently not looking at type parameters, we can statically determine the output type.
+  locally {
+    this.outputTypeTrap.dataSetType = dataSetType[WT2[_, _]]
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the CPU load of the first key extraction UDF. Currently effectless.
+   *
+   * @param udfCpuEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThisKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
+    this.keyUdf0CpuEstimator = udfCpuEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the RAM load of first the key extraction UDF. Currently effectless.
+   *
+   * @param udfRamEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThisKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
+    this.keyUdf0RamEstimator = udfRamEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the CPU load of the second key extraction UDF. Currently effectless.
+   *
+   * @param udfCpuEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThatKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
+    this.keyUdf1CpuEstimator = udfCpuEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the RAM load of the second key extraction UDF. Currently effectless.
+   *
+   * @param udfRamEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThatKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
+    this.keyUdf1RamEstimator = udfRamEstimator
+    this
+  }
+
+  /**
+   * Assemble the joined elements to new elements.
+   *
+   * @param udf produces a joined element from two joinable elements
+   * @return a new [[DataQuantaBuilder]] representing the assembled join product
+   */
+  def assemble[NewOut](udf: SerializableBiFunction[In0, In1, NewOut]) =
+    this.map(new SerializableFunction[WT2[In0, In1], NewOut] {
+      override def apply(joinTuple: WT2[In0, In1]): NewOut = udf.apply(joinTuple.field0, joinTuple.field1)
+    })
+
+  override protected def build =
+    inputDataQuanta0.dataQuanta().joinJava(keyUdf0, inputDataQuanta1.dataQuanta(), keyUdf1)(inputDataQuanta1.classTag, this.keyTag)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/KeyedDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/KeyedDataQuantaBuilder.scala
new file mode 100644
index 0000000..9ddc101
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/KeyedDataQuantaBuilder.scala
@@ -0,0 +1,50 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+
+/**
+ * This is not an actual [[DataQuantaBuilder]] but rather decorates such a [[DataQuantaBuilder]] with a key.
+ */
+class KeyedDataQuantaBuilder[Out, Key](private val dataQuantaBuilder: DataQuantaBuilder[_, Out],
+                                       private val keyExtractor: SerializableFunction[Out, Key])
+                                      (implicit javaPlanBuilder: JavaPlanBuilder) {
+
+  /**
+   * Joins this instance with the given one via their keys.
+   *
+   * @param that the instance to join with
+   * @return a [[DataQuantaBuilder]] representing the join product
+   */
+  def join[ThatOut](that: KeyedDataQuantaBuilder[ThatOut, Key]) =
+    dataQuantaBuilder.join(this.keyExtractor, that.dataQuantaBuilder, that.keyExtractor)
+
+  /**
+   * Co-groups this instance with the given one via their keys.
+   *
+   * @param that the instance to join with
+   * @return a [[DataQuantaBuilder]] representing the co-group product
+   */
+  def coGroup[ThatOut](that: KeyedDataQuantaBuilder[ThatOut, Key]) =
+    dataQuantaBuilder.coGroup(this.keyExtractor, that.dataQuantaBuilder, that.keyExtractor)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/LoadCollectionDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/LoadCollectionDataQuantaBuilder.scala
new file mode 100644
index 0000000..748416e
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/LoadCollectionDataQuantaBuilder.scala
@@ -0,0 +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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.WayangCollections
+
+import java.util.{Collection => JavaCollection}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CollectionSource]]s.
+ *
+ * @param collection      the [[JavaCollection]] to be loaded
+ * @param javaPlanBuilder the [[JavaPlanBuilder]]
+ */
+class LoadCollectionDataQuantaBuilder[Out](collection: JavaCollection[Out])(implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[LoadCollectionDataQuantaBuilder[Out], Out] {
+
+  // Try to infer the type class from the collection.
+  locally {
+    if (!collection.isEmpty) {
+      val any = WayangCollections.getAny(collection)
+      if (any != null) {
+        this.outputTypeTrap.dataSetType = DataSetType.createDefault(any.getClass)
+      }
+    }
+  }
+
+  override protected def build: DataQuanta[Out] = javaPlanBuilder.planBuilder.loadCollection(collection)(this.classTag)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapDataQuantaBuilder.scala
new file mode 100644
index 0000000..deb15a9
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapDataQuantaBuilder.scala
@@ -0,0 +1,69 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.basic.operators.MapOperator
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param udf             UDF for the [[MapOperator]]
+ */
+class MapDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In],
+                                    udf: SerializableFunction[In, Out])
+                                   (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[MapDataQuantaBuilder[In, Out], Out] {
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  // Try to infer the type classes from the udf.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In] => inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+    parameters.get("Output") match {
+      case cls: Class[Out] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+  }
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().mapJava(udf, this.udfLoadProfileEstimator)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapPartitionsDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapPartitionsDataQuantaBuilder.scala
new file mode 100644
index 0000000..e686565
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapPartitionsDataQuantaBuilder.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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapPartitionsOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param udf             UDF for the [[org.apache.wayang.basic.operators.MapPartitionsOperator]]
+ */
+class MapPartitionsDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In],
+                                              udf: SerializableFunction[java.lang.Iterable[In], java.lang.Iterable[Out]])
+                                             (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[MapPartitionsDataQuantaBuilder[In, Out], Out] {
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  /** Selectivity of the filter predicate. */
+  private var selectivity: ProbabilisticDoubleInterval = _
+
+  // Try to infer the type classes from the udf.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[In] => {
+        inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      }
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+    val originalClass = ReflectionUtils.getWrapperClass(parameters.get("Output"), 0)
+    originalClass match {
+      case cls: Class[Out] => {
+        this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      }
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+  }
+
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  /**
+   * Specify the selectivity of the UDF.
+   *
+   * @param lowerEstimate the lower bound of the expected selectivity
+   * @param upperEstimate the upper bound of the expected selectivity
+   * @param confidence    the probability of the actual selectivity being within these bounds
+   * @return this instance
+   */
+  def withSelectivity(lowerEstimate: Double, upperEstimate: Double, confidence: Double) = {
+    this.selectivity = new ProbabilisticDoubleInterval(lowerEstimate, upperEstimate, confidence)
+    this
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().mapPartitionsJava(
+    udf, this.selectivity, this.udfLoadProfileEstimator
+  )
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ProjectionDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ProjectionDataQuantaBuilder.scala
new file mode 100644
index 0000000..28ede1e
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ProjectionDataQuantaBuilder.scala
@@ -0,0 +1,37 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s with
+ * [[org.apache.wayang.basic.function.ProjectionDescriptor]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param fieldNames      field names for the [[org.apache.wayang.basic.function.ProjectionDescriptor]]
+ */
+class ProjectionDataQuantaBuilder[In, Out](inputDataQuanta: DataQuantaBuilder[_, In], fieldNames: Array[String])
+                                          (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[ProjectionDataQuantaBuilder[In, Out], Out] {
+
+  override protected def build = inputDataQuanta.dataQuanta().project(fieldNames.toSeq)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ReduceByDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ReduceByDataQuantaBuilder.scala
new file mode 100644
index 0000000..ca14576
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ReduceByDataQuantaBuilder.scala
@@ -0,0 +1,94 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction}
+import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+import scala.reflect.ClassTag
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.ReduceByOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param udf             UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
+ * @param keyUdf          key extraction UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
+ */
+class ReduceByDataQuantaBuilder[Key, T](inputDataQuanta: DataQuantaBuilder[_, T],
+                                        keyUdf: SerializableFunction[T, Key],
+                                        udf: SerializableBinaryOperator[T])
+                                       (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[ReduceByDataQuantaBuilder[Key, T], T] {
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  implicit var keyTag: ClassTag[Key] = _
+
+  /** [[LoadProfileEstimator]] to estimate the [[LoadProfile]] of the [[udf]]. */
+  private var udfLoadProfileEstimator: LoadProfileEstimator = _
+
+  // TODO: Add these estimators.
+  //  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf]]. */
+  //  private var keyUdfCpuEstimator: LoadEstimator = _
+  //
+  //  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf]]. */
+  //  private var keyUdfRamEstimator: LoadEstimator = _
+
+  // Try to infer the type classes from the UDFs.
+  locally {
+    var parameters = ReflectionUtils.getTypeParameters(udf.getClass, classOf[SerializableBinaryOperator[_]])
+    parameters.get("Type") match {
+      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", udf)
+    }
+
+    parameters = ReflectionUtils.getTypeParameters(keyUdf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[T] => this.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf)
+    }
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+
+  /**
+   * Set a [[LoadProfileEstimator]] for the load of the UDF.
+   *
+   * @param udfLoadProfileEstimator the [[LoadProfileEstimator]]
+   * @return this instance
+   */
+  def withUdfLoad(udfLoadProfileEstimator: LoadProfileEstimator) = {
+    this.udfLoadProfileEstimator = udfLoadProfileEstimator
+    this
+  }
+
+  override protected def build =
+    inputDataQuanta.dataQuanta().reduceByKeyJava(keyUdf, udf, this.udfLoadProfileEstimator)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/RepeatDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/RepeatDataQuantaBuilder.scala
new file mode 100644
index 0000000..f7170f7
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/RepeatDataQuantaBuilder.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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+import java.util.function.{Function => JavaFunction}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DoWhileOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param numRepetitions  number of repetitions of the loop
+ * @param bodyBuilder     builds the loop body
+ */
+class RepeatDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T],
+                                 numRepetitions: Int,
+                                 bodyBuilder: JavaFunction[DataQuantaBuilder[_, T], DataQuantaBuilder[_, T]])
+                                (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[RepeatDataQuantaBuilder[T], T] {
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  // TODO: We could improve by combining the TypeTraps in the body loop.
+
+  override protected def build =
+    inputDataQuanta.dataQuanta().repeat(numRepetitions, startDataQuanta => {
+      val loopStartbuilder = new FakeDataQuantaBuilder(startDataQuanta)
+      bodyBuilder(loopStartbuilder).dataQuanta()
+    })
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SampleDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SampleDataQuantaBuilder.scala
new file mode 100644
index 0000000..0642fe6
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SampleDataQuantaBuilder.scala
@@ -0,0 +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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.basic.operators.SampleOperator
+
+import java.util.function.IntUnaryOperator
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.SampleOperator]]s.
+ *
+ * @param inputDataQuanta    [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param sampleSizeFunction the absolute size of the sample as a function of the current iteration number
+ */
+class SampleDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T], sampleSizeFunction: IntUnaryOperator)
+                                (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[SampleDataQuantaBuilder[T], T] {
+
+  /**
+   * Size of the dataset to be sampled.
+   */
+  private var datasetSize = SampleOperator.UNKNOWN_DATASET_SIZE
+
+  /**
+   * Sampling method to use.
+   */
+  private var sampleMethod = SampleOperator.Methods.ANY
+
+  /**
+   * Seed to use.
+   */
+  private var seed: Option[Long] = None
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  /**
+   * Set the size of the dataset that should be sampled.
+   *
+   * @param datasetSize the size of the dataset
+   * @return this instance
+   */
+  def withDatasetSize(datasetSize: Long) = {
+    this.datasetSize = datasetSize
+    this
+  }
+
+  /**
+   * Set the sample method to be used.
+   *
+   * @param sampleMethod the sample method
+   * @return this instance
+   */
+  def withSampleMethod(sampleMethod: SampleOperator.Methods) = {
+    this.sampleMethod = sampleMethod
+    this
+  }
+
+  /**
+   * Set the sample method to be used.
+   *
+   * @param seed
+   * @return this instance
+   */
+  def withSeed(seed: Long) = {
+    this.seed = Some(seed)
+    this
+  }
+
+  override protected def build =
+    inputDataQuanta.dataQuanta().sampleDynamicJava(sampleSizeFunction, this.datasetSize, this.seed, this.sampleMethod)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SortDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SortDataQuantaBuilder.scala
new file mode 100644
index 0000000..d1af25a
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SortDataQuantaBuilder.scala
@@ -0,0 +1,97 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.util.TypeTrap
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.costs.LoadEstimator
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.ReflectionUtils
+
+import scala.reflect.ClassTag
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.SortOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ * @param keyUdf          UDF for the [[org.apache.wayang.basic.operators.SortOperator]]
+ */
+class SortDataQuantaBuilder[T, Key](inputDataQuanta: DataQuantaBuilder[_, T],
+                                    keyUdf: SerializableFunction[T, Key])
+                                   (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[SortDataQuantaBuilder[T, Key], T] {
+
+  // Reuse the input TypeTrap to enforce type equality between input and output.
+  override def getOutputTypeTrap: TypeTrap = inputDataQuanta.outputTypeTrap
+
+  /** [[ClassTag]] or surrogate of [[Key]] */
+  implicit var keyTag: ClassTag[Key] = _
+
+  /** [[LoadEstimator]] to estimate the CPU load of the [[keyUdf]]. */
+  private var keyUdfCpuEstimator: LoadEstimator = _
+
+  /** [[LoadEstimator]] to estimate the RAM load of the [[keyUdf]]. */
+  private var keyUdfRamEstimator: LoadEstimator = _
+
+
+  // Try to infer the type classes from the UDFs.
+  locally {
+    val parameters = ReflectionUtils.getTypeParameters(keyUdf.getClass, classOf[SerializableFunction[_, _]])
+    parameters.get("Input") match {
+      case cls: Class[T] => inputDataQuanta.outputTypeTrap.dataSetType = DataSetType.createDefault(cls)
+      case _ => logger.warn("Could not infer types from {}.", keyUdf)
+    }
+
+    this.keyTag = parameters.get("Output") match {
+      case cls: Class[Key] => ClassTag(cls)
+      case _ =>
+        logger.warn("Could not infer types from {}.", keyUdf)
+        ClassTag(DataSetType.none.getDataUnitType.getTypeClass)
+    }
+  }
+
+
+  /**
+   * Set a [[LoadEstimator]] for the CPU load of the first key extraction UDF. Currently effectless.
+   *
+   * @param udfCpuEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThisKeyUdfCpuEstimator(udfCpuEstimator: LoadEstimator) = {
+    this.keyUdfCpuEstimator = udfCpuEstimator
+    this
+  }
+
+  /**
+   * Set a [[LoadEstimator]] for the RAM load of first the key extraction UDF. Currently effectless.
+   *
+   * @param udfRamEstimator the [[LoadEstimator]]
+   * @return this instance
+   */
+  def withThisKeyUdfRamEstimator(udfRamEstimator: LoadEstimator) = {
+    this.keyUdfRamEstimator = udfRamEstimator
+    this
+  }
+
+  override protected def build =
+    inputDataQuanta.dataQuanta().sortJava(keyUdf)(this.keyTag)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnarySourceDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnarySourceDataQuantaBuilder.scala
new file mode 100644
index 0000000..d97b08b
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnarySourceDataQuantaBuilder.scala
@@ -0,0 +1,37 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.core.plan.wayangplan.UnarySource
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.core.plan.wayangplan.UnarySource]]s.
+ *
+ * @param source          the [[UnarySource]]
+ * @param javaPlanBuilder the [[JavaPlanBuilder]]
+ */
+class UnarySourceDataQuantaBuilder[This <: DataQuantaBuilder[_, Out], Out](source: UnarySource[Out])
+                                                                          (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[This, Out] {
+
+  override protected def build: DataQuanta[Out] = javaPlanBuilder.planBuilder.load(source)(this.classTag)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnionDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnionDataQuantaBuilder.scala
new file mode 100644
index 0000000..04fc6ac
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnionDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.UnionAllOperator]]s.
+ *
+ * @param inputDataQuanta0 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ * @param inputDataQuanta1 [[DataQuantaBuilder]] for the first input [[DataQuanta]]
+ */
+class UnionDataQuantaBuilder[T](inputDataQuanta0: DataQuantaBuilder[_, T],
+                                inputDataQuanta1: DataQuantaBuilder[_, T])
+                               (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[UnionDataQuantaBuilder[T], T] {
+
+  override def getOutputTypeTrap = inputDataQuanta0.outputTypeTrap
+
+  override protected def build = inputDataQuanta0.dataQuanta().union(inputDataQuanta1.dataQuanta())
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ZipWithIdDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ZipWithIdDataQuantaBuilder.scala
new file mode 100644
index 0000000..ded640c
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ZipWithIdDataQuantaBuilder.scala
@@ -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.wayang.api.dataquantabuilder
+
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.basic.data.{Tuple2 => WT2}
+
+/**
+ * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.ZipWithIdOperator]]s.
+ *
+ * @param inputDataQuanta [[DataQuantaBuilder]] for the input [[DataQuanta]]
+ */
+class ZipWithIdDataQuantaBuilder[T](inputDataQuanta: DataQuantaBuilder[_, T])
+                                   (implicit javaPlanBuilder: JavaPlanBuilder)
+  extends BasicDataQuantaBuilder[ZipWithIdDataQuantaBuilder[T], WT2[java.lang.Long, T]] {
+
+  // Since we are currently not looking at type parameters, we can statically determine the output type.
+  locally {
+    this.outputTypeTrap.dataSetType = dataSetType[WT2[_, _]]
+  }
+
+  override protected def build = inputDataQuanta.dataQuanta().zipWithId
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala
index 17f5e32..1eaec53 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala
@@ -18,8 +18,9 @@
 
 package org.apache.wayang.api.graph
 
+import org.apache.wayang.api.dataquantabuilder.BasicDataQuantaBuilder
 import org.apache.wayang.api.util.DataQuantaBuilderDecorator
-import org.apache.wayang.api.{BasicDataQuantaBuilder, DataQuanta, DataQuantaBuilder, JavaPlanBuilder, _}
+import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, _}
 import org.apache.wayang.basic.operators.PageRankOperator
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
 
diff --git a/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java b/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java
index 2d4bb6c..71996a0 100644
--- a/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java
+++ b/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java
@@ -18,6 +18,8 @@
 
 package org.apache.wayang.api;
 
+import org.apache.wayang.api.dataquantabuilder.GlobalReduceDataQuantaBuilder;
+import org.apache.wayang.api.dataquantabuilder.LoadCollectionDataQuantaBuilder;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
diff --git a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java
index 59b48cd..3b55a51 100644
--- a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java
+++ b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java
@@ -21,8 +21,8 @@ package org.apache.wayang.tests;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.wayang.api.JavaPlanBuilder;
-import org.apache.wayang.api.LoadCollectionDataQuantaBuilder;
-import org.apache.wayang.api.MapDataQuantaBuilder;
+import org.apache.wayang.api.dataquantabuilder.LoadCollectionDataQuantaBuilder;
+import org.apache.wayang.api.dataquantabuilder.MapDataQuantaBuilder;
 import org.apache.wayang.core.api.WayangContext;
 import org.apache.wayang.core.util.WayangArrays;
 import org.apache.wayang.java.Java;

[incubator-wayang] 03/15: [WAYANG-28] template of javadoc to sniffer package

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit e8d1d892da944b4fe76c140d8adafff1fbddcd6f
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Wed May 5 01:21:33 2021 -0400

    [WAYANG-28] template of javadoc to sniffer package
---
 .../plugin/hackit/core/sniffer/HackitSniffer.java  | 90 +++++++++++++++++++++-
 .../plugin/hackit/core/sniffer/actor/Actor.java    | 11 ++-
 .../hackit/core/sniffer/clone/BasicCloner.java     |  6 ++
 .../plugin/hackit/core/sniffer/clone/Cloner.java   | 16 +++-
 .../hackit/core/sniffer/inject/EmptyInjector.java  |  4 +
 .../hackit/core/sniffer/inject/Injector.java       | 26 ++++++-
 .../hackit/core/sniffer/shipper/PSProtocol.java    | 12 +++
 .../hackit/core/sniffer/shipper/Shipper.java       | 72 +++++++++++++++--
 .../sniffer/shipper/receiver/BufferReceiver.java   |  4 +
 .../sniffer/shipper/receiver/EmptyReceiver.java    |  4 +
 .../core/sniffer/shipper/receiver/Receiver.java    | 17 ++++
 .../core/sniffer/shipper/sender/EmptySender.java   |  8 +-
 .../hackit/core/sniffer/shipper/sender/Sender.java | 20 ++++-
 .../core/sniffer/sniff/CollectionTagsToSniff.java  | 18 +++++
 .../core/sniffer/sniff/SingleTagToSniff.java       | 15 ++++
 .../plugin/hackit/core/sniffer/sniff/Sniff.java    | 11 ++-
 16 files changed, 313 insertions(+), 21 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java
index 02342ff..6740265 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/HackitSniffer.java
@@ -30,19 +30,75 @@ import java.io.Serializable;
 import java.util.Iterator;
 import java.util.function.Function;
 
-public class HackitSniffer<K, T, SentType, SenderObj extends Sender<SentType>, ReceiverObj extends Receiver<HackitTuple<K,T>> > implements Function<HackitTuple<K, T>, Iterator<HackitTuple<K, T>>>, Serializable {
-
+/**
+ *
+ * @param <K>
+ * @param <T>
+ * @param <SentType>
+ * @param <SenderObj>
+ * @param <ReceiverObj>
+ */
+public class
+    HackitSniffer<
+        K,
+        T,
+        SentType,
+        SenderObj extends Sender<SentType>,
+        ReceiverObj extends Receiver<HackitTuple<K,T>>
+    >
+    implements
+        Function<
+            HackitTuple<K, T>,
+            Iterator<HackitTuple<K, T>>
+        >,
+        Serializable {
+
+    /**
+     *
+     */
     private transient boolean not_first = false;
+
+    /**
+     *
+     */
     private Injector<HackitTuple<K, T>> hackItInjector;
 
+    /**
+     *
+     */
     private Actor<HackitTuple<K, T>> actorFunction;
 
+    /**
+     *
+     */
     private Shipper<HackitTuple<K, T>, SentType, SenderObj, ReceiverObj> shipper;
 
+    /**
+     *
+     */
     private Sniff<HackitTuple<K, T>> hackItSniff;
+
+    /**
+     *
+     */
     private Cloner<HackitTuple<K, T>, SentType> hackItCloner;
 
-    public HackitSniffer(Injector<HackitTuple<K, T>> hackItInjector, Actor<HackitTuple<K, T>> actorFunction, Shipper<HackitTuple<K, T>, SentType, SenderObj, ReceiverObj> shipper, Sniff<HackitTuple<K, T>> hackItSniff, Cloner<HackitTuple<K, T>, SentType> hackItCloner) {
+    /**
+     *
+     * @param hackItInjector
+     * @param actorFunction
+     * @param shipper
+     * @param hackItSniff
+     * @param hackItCloner
+     */
+    //TODO: it may private, because need to be executed just at the creation moment
+    public HackitSniffer(
+            Injector<HackitTuple<K, T>> hackItInjector,
+            Actor<HackitTuple<K, T>> actorFunction,
+            Shipper<HackitTuple<K, T>, SentType, SenderObj, ReceiverObj> shipper,
+            Sniff<HackitTuple<K, T>> hackItSniff,
+            Cloner<HackitTuple<K, T>, SentType> hackItCloner
+    ) {
         this.hackItInjector = hackItInjector;
         this.actorFunction = actorFunction;
         this.shipper = shipper;
@@ -51,6 +107,9 @@ public class HackitSniffer<K, T, SentType, SenderObj extends Sender<SentType>, R
         this.not_first = false;
     }
 
+    /**
+     * Default Construct, this get all the components from configuration files
+     */
     public HackitSniffer() {
         //TODO this over configuration file
         this.not_first = false;
@@ -76,26 +135,51 @@ public class HackitSniffer<K, T, SentType, SenderObj extends Sender<SentType>, R
         return this.hackItInjector.inject(ktHackItTuple, inyection);
     }
 
+    /**
+     *
+     * @param hackItInjector
+     * @return
+     */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setHackItInjector(Injector<HackitTuple<K, T>> hackItInjector) {
         this.hackItInjector = hackItInjector;
         return this;
     }
 
+    /**
+     *
+     * @param actorFunction
+     * @return
+     */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setActorFunction(Actor<HackitTuple<K, T>> actorFunction) {
         this.actorFunction = actorFunction;
         return this;
     }
 
+    /**
+     *
+     * @param shipper
+     * @return
+     */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setShipper(Shipper<HackitTuple<K, T>, SentType, SenderObj, ReceiverObj> shipper) {
         this.shipper = shipper;
         return this;
     }
 
+    /**
+     *
+     * @param hackItSniff
+     * @return
+     */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setHackItSniff(Sniff<HackitTuple<K, T>> hackItSniff) {
         this.hackItSniff = hackItSniff;
         return this;
     }
 
+    /**
+     *
+     * @param hackItCloner
+     * @return
+     */
     public HackitSniffer<K, T, SentType, SenderObj, ReceiverObj> setHackItCloner(Cloner<HackitTuple<K, T>, SentType> hackItCloner) {
         this.hackItCloner = hackItCloner;
         return this;
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java
index b49a7c4..0deffd8 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/actor/Actor.java
@@ -19,7 +19,16 @@ package org.apache.wayang.plugin.hackit.core.sniffer.actor;
 
 import java.io.Serializable;
 
+/**
+ *
+ * @param <T>
+ */
 public interface Actor<T> extends Serializable {
 
-    public boolean is_sendout(T value);
+    /**
+     *
+     * @param value
+     * @return
+     */
+    boolean is_sendout(T value);
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/BasicCloner.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/BasicCloner.java
index c644e33..71c5255 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/BasicCloner.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/BasicCloner.java
@@ -21,7 +21,13 @@ import org.apache.commons.lang3.SerializationUtils;
 
 import java.io.Serializable;
 
+/**
+ * BasicClonner is just a basic implentation of {@link Cloner}
+ *
+ * @param <I> type of element to clone
+ */
 public class BasicCloner<I> implements Cloner<I, byte[]> {
+
     @Override
     public byte[] clone(I input) {
         return SerializationUtils.serialize((Serializable) input);
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/Cloner.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/Cloner.java
index ccf99ab..74e2eb3 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/Cloner.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/clone/Cloner.java
@@ -19,7 +19,21 @@ package org.apache.wayang.plugin.hackit.core.sniffer.clone;
 
 import java.io.Serializable;
 
+/**
+ * Cloner is the template for the functionality that take care about the clone phase
+ *
+ * @param <I> type of the element that it will get cloned
+ * @param <O> type of the element after cloning it, it possible that could be different
+ */
 public interface Cloner<I, O> extends Serializable {
 
-    public O clone(I input);
+    /**
+     * get <code>input</code> and create a clone, the output cloud be different
+     * to the original, this doesn't change the behavior it just a to reduce overhead
+     * in some case that is not need to have the same kind.
+     *
+     * @param input element to get cloned
+     * @return Clone of the <code>input</code> element
+     */
+    O clone(I input);
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java
index 94d265f..cacca18 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/EmptyInjector.java
@@ -24,6 +24,10 @@ import java.util.Spliterator;
 import java.util.Spliterators;
 import java.util.stream.StreamSupport;
 
+/**
+ *
+ * @param <T>
+ */
 public class EmptyInjector<T> implements Injector<T>{
 
     @Override
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java
index 3ca4563..fb26604 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/inject/Injector.java
@@ -20,11 +20,31 @@ package org.apache.wayang.plugin.hackit.core.sniffer.inject;
 import java.io.Serializable;
 import java.util.Iterator;
 
+/**
+ *
+ * @param <T>
+ */
 public interface Injector<T> extends Serializable {
 
-    public Iterator<T> inject(T element, Iterator<T> iterator);
+    /**
+     *
+     * @param element
+     * @param iterator
+     * @return
+     */
+    Iterator<T> inject(T element, Iterator<T> iterator);
 
-    public boolean is_skip_element(T element);
+    /**
+     *
+     * @param element
+     * @return
+     */
+    boolean is_skip_element(T element);
 
-    public boolean is_halt_job(T element);
+    /**
+     *
+     * @param element
+     * @return
+     */
+    boolean is_halt_job(T element);
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java
index 6a19794..703074c 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/PSProtocol.java
@@ -21,6 +21,18 @@ package org.apache.wayang.plugin.hackit.core.sniffer.shipper;
  * Publish and Subscribed Protocol
  */
 public interface PSProtocol {
+
+    /**
+     *
+     * @param topic
+     * @return
+     */
     public PSProtocol addTopic(String... topic);
+
+    /**
+     *
+     * @param exchange
+     * @return
+     */
     public PSProtocol addExchange(String exchange);
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java
index 969d8e8..a9881af 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/Shipper.java
@@ -23,15 +23,41 @@ import org.apache.wayang.plugin.hackit.core.sniffer.shipper.sender.Sender;
 import java.io.Serializable;
 import java.util.Iterator;
 
+/**
+ *
+ * @param <T>
+ * @param <ST>
+ * @param <SenderObj>
+ * @param <ReceiverObj>
+ */
 public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj extends Receiver<T>> implements Iterator<T>, Serializable {
 
+    /**
+     *
+     */
     protected Sender sender_instance;
+
+    /**
+     *
+     */
     protected Receiver receiver_instance;
 
+    /**
+     *
+     * @return
+     */
     protected abstract Sender createSenderInstance();
+
+    /**
+     *
+     * @return
+     */
     protected abstract Receiver createReceiverInstance();
 
-    /** Connect with the a Message queue service*/
+    /**
+     * Connect with the a Message queue service
+     * @param value
+     */
     public void publish(ST value){
         if(this.sender_instance == null){
             throw new RuntimeException("The Sender of the Shipper is not instanciated");
@@ -39,16 +65,27 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
         this.sender_instance.send(value);
     }
 
-    /** To subscribe as a producer */
+    /**
+     * To subscribe as a producer
+     */
     public void subscribeAsProducer(){
         this.sender_instance = this.createSenderInstance();
         this.sender_instance.init();
     }
 
+    /**
+     *
+     * @param topic
+     */
     public void subscribeAsProducer(String... topic){
         this.subscribeAsProducer("default", topic);
     }
 
+    /**
+     *
+     * @param metatopic
+     * @param topic
+     */
     public void subscribeAsProducer(String metatopic, String... topic){
         this.subscribeAsProducer();
         ((PSProtocol)this.sender_instance)
@@ -57,23 +94,37 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
         ;
     }
 
-    /** Close connection */
+    /**
+     * Close connection
+     */
     public void unsubscribeAsProducer(){
         if( this.sender_instance == null) return;
         this.sender_instance.close();
     }
 
-    /** To subscribe/unsubscribe as a consumer
+    /**
+     * To subscribe/unsubscribe as a consumer
      * metatopic correspond to EXCHANGE_NAME
      * topics correspond to bindingKeys
-     * */
+     */
     public void subscribeAsConsumer(){
         this.receiver_instance = this.createReceiverInstance();
         this.receiver_instance.init();
     }
+
+    /**
+     *
+     * @param topic
+     */
     public void subscribeAsConsumer(String... topic){
         this.subscribeAsProducer("default", topic);
     }
+
+    /**
+     *
+     * @param metatopic
+     * @param topic
+     */
     public void subscribeAsConsumer(String metatopic, String... topic){
         this.subscribeAsConsumer();
         ((PSProtocol)this.receiver_instance)
@@ -82,12 +133,17 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
         ;
     }
 
-    /** Close connection */
+    /**
+     * Close connection
+     */
     public void unsubscribeAsConsumer() {
         if( this.receiver_instance == null) return;
         this.receiver_instance.close();
     }
 
+    /**
+     *
+     */
     public void close(){
         this.unsubscribeAsConsumer();
         this.unsubscribeAsProducer();
@@ -99,6 +155,10 @@ public abstract class Shipper<T, ST, SenderObj extends Sender<ST>, ReceiverObj e
     @Override
     public abstract T next();
 
+    /**
+     *
+     * @return
+     */
     public Iterator<T> getNexts(){
         if( this.receiver_instance == null){
             throw new RuntimeException("The Receiver of the Shipper is not instanciated");
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java
index 0a93dae..a13cabb 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/BufferReceiver.java
@@ -21,6 +21,10 @@ import java.io.Serializable;
 import java.util.Queue;
 import java.util.concurrent.LinkedBlockingQueue;
 
+/**
+ *
+ * @param <T>
+ */
 public class BufferReceiver<T> implements Serializable {
     //TODO implement the doble buffering
     private transient Queue<T> queue;
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java
index 2bd42a7..fdfe139 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/EmptyReceiver.java
@@ -21,6 +21,10 @@ package org.apache.wayang.plugin.hackit.core.sniffer.shipper.receiver;
 import java.util.Collections;
 import java.util.Iterator;
 
+/**
+ *
+ * @param <T>
+ */
 public class EmptyReceiver<T> extends Receiver<T> {
 
     @Override
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java
index dd120c0..80f9a9d 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/receiver/Receiver.java
@@ -20,13 +20,30 @@ package org.apache.wayang.plugin.hackit.core.sniffer.shipper.receiver;
 import java.io.Serializable;
 import java.util.Iterator;
 
+/**
+ *
+ * @param <T>
+ */
 public abstract class Receiver<T> implements Serializable {
 
+    /**
+     *
+     */
     private transient BufferReceiver<T> bufferReceive;
 
+    /**
+     *
+     */
     public abstract void init();
 
+    /**
+     *
+     * @return
+     */
     public abstract Iterator<T> getElements();
 
+    /**
+     *
+     */
     public abstract void close();
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java
index 6ac0817..1a2c459 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/EmptySender.java
@@ -18,12 +18,14 @@
 
 package org.apache.wayang.plugin.hackit.core.sniffer.shipper.sender;
 
+/**
+ *
+ * @param <T>
+ */
 public class EmptySender<T> implements Sender<T> {
 
     @Override
-    public void init() {
-
-    }
+    public void init() {}
 
     @Override
     public void send(T value) {}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java
index 89eb57d..82a491c 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/shipper/sender/Sender.java
@@ -19,11 +19,25 @@ package org.apache.wayang.plugin.hackit.core.sniffer.shipper.sender;
 
 import java.io.Serializable;
 
+/**
+ *
+ * @param <T>
+ */
 public interface Sender<T> extends Serializable {
 
-    public void init();
+    /**
+     *
+     */
+    void init();
 
-    public void send(T value);
+    /**
+     *
+     * @param value
+     */
+    void send(T value);
 
-    public void close();
+    /**
+     *
+     */
+    void close();
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java
index a2811b0..16419db 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/CollectionTagsToSniff.java
@@ -23,18 +23,36 @@ import org.apache.wayang.plugin.hackit.core.tags.HackitTag;
 import java.util.HashSet;
 import java.util.Set;
 
+/**
+ *
+ */
 public class CollectionTagsToSniff implements Sniff {
 
+    /**
+     *
+     */
     public Set<HackitTag> tags2sniff;
 
+    /**
+     * Default Construct
+     */
     public CollectionTagsToSniff(){
         this.tags2sniff = new HashSet<>();
     }
 
+    /**
+     *
+     * @param tag
+     * @return
+     */
     public boolean sniff(HackitTag tag){
         return this.tags2sniff.contains(tag);
     }
 
+    /**
+     *
+     * @param tag
+     */
     public void addTag2sniff(HackitTag tag) {
         this.tags2sniff.add(tag);
     }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java
index 9ce8ede..42da261 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/SingleTagToSniff.java
@@ -20,14 +20,29 @@ package org.apache.wayang.plugin.hackit.core.sniffer.sniff;
 
 import org.apache.wayang.plugin.hackit.core.tags.HackitTag;
 
+/**
+ *
+ */
 public class SingleTagToSniff implements Sniff {
 
+    /**
+     *
+     */
     public HackitTag tags2sniff;
 
+    /**
+     *
+     * @param tag
+     * @return
+     */
     public boolean sniff(HackitTag tag){
         return this.tags2sniff.equals(tag);
     }
 
+    /**
+     *
+     * @param tag
+     */
     public void addTag2sniff(HackitTag tag) {
         if(this.tags2sniff != null){
             throw new RuntimeException("The SingleTagToSniff already got the tag, if you need more of one tag use CollectionTagsToSniff class");
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java
index 44bfbbb..3a57c6f 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/sniffer/sniff/Sniff.java
@@ -19,7 +19,16 @@ package org.apache.wayang.plugin.hackit.core.sniffer.sniff;
 
 import java.io.Serializable;
 
+/**
+ *
+ * @param <I>
+ */
 public interface Sniff<I> extends Serializable {
 
-    public boolean sniff(I input);
+    /**
+     *
+     * @param input
+     * @return
+     */
+    boolean sniff(I input);
 }

[incubator-wayang] 12/15: [WAYANG-28] update the structure of class in the Tagger function wrappers

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit ad7f4510c8cedb22e6d3bffaebbe7229f5f9d244
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Mon May 17 12:40:25 2021 -0400

    [WAYANG-28] update the structure of class in the Tagger function wrappers
---
 .../core/tagger/wrapper/FlatmapWrapperHackit.java  | 10 +++---
 .../core/tagger/wrapper/FunctionWrapperHackit.java | 13 ++++---
 .../tagger/wrapper/PredicateWrapperHackit.java     | 12 +++----
 .../tagger/wrapper/template/FlatMapTemplate.java   | 40 ---------------------
 .../template/TaggerWrapperFlatMapTemplate.java     | 41 ++++++++++++++++++++++
 ...ate.java => TaggerWrapperFunctionTemplate.java} |  4 +--
 6 files changed, 60 insertions(+), 60 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
index 9100c99..5190f26 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
@@ -18,13 +18,13 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FlatMapTemplate;
+import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.TaggerWrapperFlatMapTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
 import java.util.Iterator;
 
 /**
- * FlatmapWrapperHackit is an implementation of {@link FlatMapTemplate} where Hackit manage the logic
+ * FlatmapWrapperHackit is an implementation of {@link TaggerWrapperFlatMapTemplate} where Hackit manage the logic
  * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
  * original function
  *
@@ -34,19 +34,19 @@ import java.util.Iterator;
  */
 public class FlatmapWrapperHackit<IDType, I, O>
         extends HackitTagger
-        implements FlatMapTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
+        implements TaggerWrapperFlatMapTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
 
     /**
      * Original function that will transform the data
      */
-    private FlatMapTemplate<I, O> function;
+    private TaggerWrapperFlatMapTemplate<I, O> function;
 
     /**
      * Default Construct
      *
      * @param function is the function that will be Wrapped by the {@link FlatmapWrapperHackit}
      */
-    public FlatmapWrapperHackit( FlatMapTemplate<I, O> function ) {
+    public FlatmapWrapperHackit(TaggerWrapperFlatMapTemplate<I, O> function ) {
         this.function = function;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
index 99e4b37..982b838 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
@@ -18,12 +18,11 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FlatMapTemplate;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FunctionTemplate;
+import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.TaggerWrapperFunctionTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
 /**
- * FunctionWrapperHackit is an implementation of {@link FunctionTemplate} where Hackit manage the logic
+ * FunctionWrapperHackit is an implementation of {@link TaggerWrapperFunctionTemplate} where Hackit manage the logic
  * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
  * original function
  *
@@ -31,21 +30,21 @@ import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
  * @param <I> Input Type of the original Tuple
  * @param <O> Output Type after the perform the Function
  */
-public class FunctionWrapperHackit <IDType, I, O>
+public class FunctionWrapperHackit<IDType, I, O>
         extends HackitTagger
-        implements FunctionTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
+        implements TaggerWrapperFunctionTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
 
     /**
      * Original function that will transform the data
      */
-    private FunctionTemplate<I, O> function;
+    private TaggerWrapperFunctionTemplate<I, O> function;
 
     /**
      * Default Construct
      *
      * @param function is the function that will be Wrapped by the {@link FunctionWrapperHackit}
      */
-    public FunctionWrapperHackit(FunctionTemplate<I, O> function) {
+    public FunctionWrapperHackit(TaggerWrapperFunctionTemplate<I, O> function) {
         this.function = function;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
index 7249fdf..f9d47cd 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
@@ -18,13 +18,13 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FunctionTemplate;
+import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.TaggerWrapperFunctionTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
 /**
- * PredicateWrapperHackit is an implementation of {@link FunctionTemplate} where Hackit manage the logic
+ * PredicateWrapperHackit is an implementation of {@link TaggerWrapperFunctionTemplate} where Hackit manage the logic
  * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
- * original function. The original {@link FunctionTemplate} it an predicate function because return a
+ * original function. The original {@link TaggerWrapperFunctionTemplate} it an predicate function because return a
  * {@link Boolean}
  *
  * @param <IDType> Type of {@link org.apache.wayang.plugin.hackit.core.tuple.header.Header} key of the {@link HackitTuple}
@@ -32,19 +32,19 @@ import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
  */
 public class PredicateWrapperHackit<IDType, I>
         extends HackitTagger
-        implements FunctionTemplate<HackitTuple<IDType, I>, Boolean> {
+        implements TaggerWrapperFunctionTemplate<HackitTuple<IDType, I>, Boolean> {
 
     /**
      * Original predicate that will evaluate the data to give a True or False value
      */
-    private FunctionTemplate<I, Boolean> function;
+    private TaggerWrapperFunctionTemplate<I, Boolean> function;
 
     /**
      * Default Construct
      *
      * @param function is the predicate that will be Wrapped by the {@link PredicateWrapperHackit}
      */
-    public PredicateWrapperHackit(FunctionTemplate<I, Boolean> function) {
+    public PredicateWrapperHackit(TaggerWrapperFunctionTemplate<I, Boolean> function) {
         this.function = function;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.java
deleted file mode 100644
index 6c68f0c..0000000
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.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.wayang.plugin.hackit.core.tagger.wrapper.template;
-
-import java.util.Iterator;
-
-/**
- * FlatMapTemplate is the template that provide the abstraction to work with Flatmap operations and also
- * allows to wrap some function made by the user.
- *
- * FlatMapTemplate generate as output a {@link Iterator} this could be an extension of {@link org.apache.wayang.plugin.hackit.core.iterator.HackitIterator}
- *
- * @param <I> Input type of the original Function
- * @param <O> Output type of the original function
- */
-public interface FlatMapTemplate<I, O> {
-
-    /**
-     * Execute the logic over one element and generate as output a {@link Iterator}
-     *
-     * @param input element to transform
-     * @return {@link Iterator} that contains the output's
-     */
-    public Iterator<O> execute(I input);
-}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFlatMapTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFlatMapTemplate.java
new file mode 100644
index 0000000..bb05c3a
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFlatMapTemplate.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.wayang.plugin.hackit.core.tagger.wrapper.template;
+
+import java.util.Iterator;
+
+/**
+ * TaggerWrapperFlatMapTemplate is the template that provide the abstraction to work with Flatmap operations and also
+ * allows to wrap some function made by the user.
+ *
+ * TaggerWrapperFlatMapTemplate generate as output a {@link Iterator} this could be an extension of {@link org.apache.wayang.plugin.hackit.core.iterator.HackitIterator}
+ *
+ * @param <I> Input type of the original Function
+ * @param <O> Output type of the original function
+ */
+public interface TaggerWrapperFlatMapTemplate<I, O> extends TaggerWrapperFunctionTemplate<I, Iterator<O>> {
+
+    /**
+     * Execute the logic over one element and generate as output a {@link Iterator}
+     *
+     * @param input element to transform
+     * @return {@link Iterator} that contains the output's
+     */
+    Iterator<O> execute(I input);
+}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFunctionTemplate.java
similarity index 87%
rename from wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java
rename to wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFunctionTemplate.java
index e33ad77..1033d4f 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFunctionTemplate.java
@@ -18,13 +18,13 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper.template;
 
 /**
- * FunctionTemplate is the template that provide the abstraction to work with Transformation operations and also
+ * TaggerWrapperFunctionTemplate is the template that provide the abstraction to work with Transformation operations and also
  * allows to wrap some function made by the user.
  *
  * @param <I> Input type of the original Function
  * @param <O> Output type of the original function
  */
-public interface FunctionTemplate<I, O> {
+public interface TaggerWrapperFunctionTemplate<I, O> {
 
     /**
      * Execute the logic over one element and generate as output <code>T</code>

[incubator-wayang] 14/15: [WAYANG-28] seed version of the Hackit API

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit a272d73203333bd4b141e923e525bc6df6c635e1
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Tue May 18 00:50:30 2021 -0400

    [WAYANG-28] seed version of the Hackit API
---
 .../apache/wayang/api/dataquanta/DataQuanta.scala  |   2 +-
 .../wayang-hackit/wayang-hackit-api/pom.xml        |   5 +
 .../plugin/hackit/api/DataQuantaHackit.scala       | 226 +++++++++++++--------
 .../apache/wayang/plugin/hackit/api/Hackit.scala   |  78 +++++++
 .../plugin/hackit/api/ApiExtensionTest.scala       |  42 +++-
 5 files changed, 260 insertions(+), 93 deletions(-)

diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
index 664079b..4f9eb43 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
@@ -49,7 +49,7 @@ import scala.reflect._
   * @param ev$1        the data type of the elements in this instance
   * @param planBuilder keeps track of the [[WayangPlan]] being build
   */
-abstract class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: Int = 0)(implicit val planBuilder: PlanBuilder) {
+abstract class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, var outputIndex: Int = 0)(implicit val planBuilder: PlanBuilder) {
 
   Validate.isTrue(operator.getNumOutputs > outputIndex)
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
index ae95199..85a37b7 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
@@ -50,6 +50,11 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.wayang</groupId>
+            <artifactId>wayang-hackit-core</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.wayang</groupId>
             <artifactId>wayang-api-scala-java_2.11</artifactId>
             <version>0.6.0-SNAPSHOT</version>
         </dependency>
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala
index 3f76c7b..f1505e0 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/DataQuantaHackit.scala
@@ -29,10 +29,17 @@ import org.apache.wayang.core.function.{FlatMapDescriptor, MapPartitionsDescript
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
 import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
 import org.apache.wayang.core.plan.wayangplan.{ElementaryOperator, Operator, OutputSlot, WayangPlan}
+import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger
+import org.apache.wayang.plugin.hackit.core.tagger.wrapper.FunctionWrapperHackit
+import org.apache.wayang.plugin.hackit.core.tags.HackitTag
+import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple
 
-import java.lang
+
+
+import java.{lang, util}
 import java.lang.{Iterable => JavaIterable}
 import java.util.function.IntUnaryOperator
+import scala.language.implicitConversions
 import scala.reflect.ClassTag
 
 /**
@@ -42,10 +49,35 @@ import scala.reflect.ClassTag
  * @param ev$1        the data type of the elements in this instance
  * @param planBuilder keeps track of the [[WayangPlan]] being build
  */
-class DataQuantaHackit[Out: ClassTag]
+class DataQuantaHackit[Out : ClassTag]
                             (override val operator: ElementaryOperator, outputIndex: Int = 0)
                             (implicit override val  planBuilder: PlanBuilder)
-                    extends DataQuanta[Out](operator, outputIndex) {
+                    extends DataQuanta[Out](operator, outputIndex)
+{
+
+  var tagger : HackitTagger = null;
+  //var sniffer : HackitSniffer = null;
+
+  /**
+   * add a [[HackitTag]] on the [[HackitTagger]] to enable the process
+   *
+   * @param tag [[HackitTag]] to be added
+   * @return the self instance
+   */
+  //TODO add the version of collection
+  def addTag(tag: HackitTag): DataQuantaHackit[Out] = {
+//    tagger.addPreTag(tag)
+    println("here")
+    this
+  }
+
+  def underHackit[Key: ClassTag](): DataQuantaHackit[HackitTuple[Key, Out]] = {
+//    HackItRDD<K, T> ktHackItRDD = new HackItRDD<K, T>(
+//      rdd.map((Function<T, HackItTuple<K, T>>) HackItTuple::new)
+//    );
+
+null
+  }
 
   /**
    * Feed this instance into a [[MapOperator]].
@@ -54,16 +86,21 @@ class DataQuantaHackit[Out: ClassTag]
    * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
    * @return a new instance representing the [[MapOperator]]'s output
    */
-  override def mapJava[NewOut: ClassTag](udf: SerializableFunction[Out, NewOut],
+   override def mapJava[NewOut: ClassTag](udf: SerializableFunction[Out, NewOut],
                                          udfLoad: LoadProfileEstimator = null): DataQuantaHackit[NewOut] = {
-    val lala = new SerializableFunction[Out, NewOut] {
-      override def apply(t: Out): NewOut = {
+
+    val wrapper_internal = new FunctionWrapperHackit[Object, Out, NewOut](udf)
+    this.tagger = wrapper_internal
+
+    val wrapper = new SerializableFunction[HackitTuple[Object, Out], HackitTuple[Object, NewOut]](){
+      override def apply(t: HackitTuple[Object, Out]): HackitTuple[Object, NewOut] = {
         println(t)
-        udf.apply(t)
+        wrapper_internal.apply(t)
       }
     }
+
     val mapOperator = new MapOperator(new TransformationDescriptor(
-      lala, basicDataUnitType[Out], basicDataUnitType[NewOut], udfLoad
+      wrapper, basicDataUnitType[HackitTuple[Object, Out]], basicDataUnitType[HackitTuple[Object, NewOut]], udfLoad
     ))
     this.connectTo(mapOperator, 0)
     DataQuantaHackit.wrap[NewOut](mapOperator)
@@ -114,11 +151,12 @@ class DataQuantaHackit[Out: ClassTag]
                           sqlUdf: String = null,
                           selectivity: ProbabilisticDoubleInterval = null,
                           udfLoad: LoadProfileEstimator = null): DataQuantaHackit[Out] = {
-    val filterOperator = new FilterOperator(new PredicateDescriptor(
-      udf, this.output.getType.getDataUnitType.toBasicDataUnitType, selectivity, udfLoad
-    ).withSqlImplementation(sqlUdf))
-    this.connectTo(filterOperator, 0)
-    DataQuantaHackit.wrap[Out](filterOperator)
+//    val filterOperator = new FilterOperator(new PredicateDescriptor(
+//      udf, this.output.getType.getDataUnitType.toBasicDataUnitType, selectivity, udfLoad
+//    ).withSqlImplementation(sqlUdf))
+//    this.connectTo(filterOperator, 0)
+//    DataQuantaHackit.wrap[Out](filterOperator)
+    null
   }
 
   /**
@@ -132,11 +170,12 @@ class DataQuantaHackit[Out: ClassTag]
   override def flatMapJava[NewOut: ClassTag](udf: SerializableFunction[Out, JavaIterable[NewOut]],
                                              selectivity: ProbabilisticDoubleInterval = null,
                                              udfLoad: LoadProfileEstimator = null): DataQuantaHackit[NewOut] = {
-    val flatMapOperator = new FlatMapOperator(new FlatMapDescriptor(
-      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad
-    ))
-    this.connectTo(flatMapOperator, 0)
-    DataQuantaHackit.wrap[NewOut](flatMapOperator)
+//    val flatMapOperator = new FlatMapOperator(new FlatMapDescriptor(
+//      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad
+//    ))
+//    this.connectTo(flatMapOperator, 0)
+//    DataQuantaHackit.wrap[NewOut](flatMapOperator)
+    null
   }
 
 
@@ -199,12 +238,13 @@ class DataQuantaHackit[Out: ClassTag]
                                               udf: SerializableBinaryOperator[Out],
                                               udfLoad: LoadProfileEstimator = null)
   : DataQuantaHackit[Out] = {
-    val reduceByOperator = new ReduceByOperator(
-      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
-      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
-    )
-    this.connectTo(reduceByOperator, 0)
-    DataQuantaHackit.wrap[Out](reduceByOperator)
+//    val reduceByOperator = new ReduceByOperator(
+//      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+//      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
+//    )
+//    this.connectTo(reduceByOperator, 0)
+//    DataQuantaHackit.wrap[Out](reduceByOperator)
+    null
   }
 
   /**
@@ -216,13 +256,14 @@ class DataQuantaHackit[Out: ClassTag]
    */
   override def groupByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
                                              keyUdfLoad: LoadProfileEstimator = null): DataQuantaHackit[java.lang.Iterable[Out]] = {
-    val groupByOperator = new MaterializedGroupByOperator(
-      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key], keyUdfLoad),
-      dataSetType[Out],
-      groupedDataSetType[Out]
-    )
-    this.connectTo(groupByOperator, 0)
-    DataQuantaHackit.wrap[java.lang.Iterable[Out]](groupByOperator)
+//    val groupByOperator = new MaterializedGroupByOperator(
+//      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key], keyUdfLoad),
+//      dataSetType[Out],
+//      groupedDataSetType[Out]
+//    )
+//    this.connectTo(groupByOperator, 0)
+//    DataQuantaHackit.wrap[java.lang.Iterable[Out]](groupByOperator)
+    null
   }
 
   /**
@@ -234,11 +275,12 @@ class DataQuantaHackit[Out: ClassTag]
    */
   override def reduceJava(udf: SerializableBinaryOperator[Out],
                           udfLoad: LoadProfileEstimator = null): DataQuantaHackit[Out] = {
-    val globalReduceOperator = new GlobalReduceOperator(
-      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
-    )
-    this.connectTo(globalReduceOperator, 0)
-    DataQuantaHackit.wrap[Out](globalReduceOperator)
+//    val globalReduceOperator = new GlobalReduceOperator(
+//      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
+//    )
+//    this.connectTo(globalReduceOperator, 0)
+//    DataQuantaHackit.wrap[Out](globalReduceOperator)
+    null
   }
 
   /**
@@ -250,14 +292,15 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[JoinOperator]]'s output
    */
   override def joinJava[ThatOut: ClassTag, Key: ClassTag](thisKeyUdf: SerializableFunction[Out, Key], that: DataQuanta[ThatOut], thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuantaHackit[WayangTuple2[Out, ThatOut]] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val joinOperator = new JoinOperator(
-      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
-      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
-    )
-    this.connectTo(joinOperator, 0)
-    that.connectTo(joinOperator, 1)
-    DataQuantaHackit.wrap[WayangTuple2[Out, ThatOut]](joinOperator)
+//    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+//    val joinOperator = new JoinOperator(
+//      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+//      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
+//    )
+//    this.connectTo(joinOperator, 0)
+//    that.connectTo(joinOperator, 1)
+//    DataQuantaHackit.wrap[WayangTuple2[Out, ThatOut]](joinOperator)
+    null
   }
 
   /**
@@ -269,14 +312,15 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[CoGroupOperator]]'s output
    */
   override def coGroupJava[ThatOut: ClassTag, Key: ClassTag](thisKeyUdf: SerializableFunction[Out, Key], that: DataQuanta[ThatOut], thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuantaHackit[WayangTuple2[JavaIterable[Out], JavaIterable[ThatOut]]] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val coGroupOperator = new CoGroupOperator(
-      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
-      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
-    )
-    this.connectTo(coGroupOperator, 0)
-    that.connectTo(coGroupOperator, 1)
-    DataQuantaHackit.wrap[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]](coGroupOperator)
+//    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+//    val coGroupOperator = new CoGroupOperator(
+//      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+//      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
+//    )
+//    this.connectTo(coGroupOperator, 0)
+//    that.connectTo(coGroupOperator, 1)
+//    DataQuantaHackit.wrap[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]](coGroupOperator)
+    null
   }
 
 
@@ -286,13 +330,12 @@ class DataQuantaHackit[Out: ClassTag]
    * @param keyUdf UDF to extract key from data quanta in this instance
    * @return a new instance representing the [[SortOperator]]'s output
    */
-  override def sortJava[Key: ClassTag]
-  (keyUdf: SerializableFunction[Out, Key])
-  : DataQuantaHackit[Out] = {
-    val sortOperator = new SortOperator(new TransformationDescriptor(
-      keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]))
-    this.connectTo(sortOperator, 0)
-    DataQuantaHackit.wrap[Out](sortOperator)
+  override def sortJava[Key: ClassTag] (keyUdf: SerializableFunction[Out, Key]) : DataQuantaHackit[Out] = {
+//    val sortOperator = new SortOperator(new TransformationDescriptor(
+//      keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]))
+//    this.connectTo(sortOperator, 0)
+//    DataQuantaHackit.wrap[Out](sortOperator)
+     null
   }
 
 
@@ -302,9 +345,10 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[GlobalMaterializedGroupOperator]]'s output
    */
   override def group(): DataQuantaHackit[JavaIterable[Out]] = {
-    val groupOperator = new GlobalMaterializedGroupOperator(dataSetType[Out], groupedDataSetType[Out])
-    this.connectTo(groupOperator, 0)
-    DataQuantaHackit.wrap[JavaIterable[Out]](groupOperator)
+//    val groupOperator = new GlobalMaterializedGroupOperator(dataSetType[Out], groupedDataSetType[Out])
+//    this.connectTo(groupOperator, 0)
+//    DataQuantaHackit.wrap[JavaIterable[Out]](groupOperator)
+    null
   }
 
   /**
@@ -314,11 +358,12 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[UnionAllOperator]]'s output
    */
   override def union(that: DataQuanta[Out]): DataQuantaHackit[Out] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val unionAllOperator = new UnionAllOperator(dataSetType[Out])
-    this.connectTo(unionAllOperator, 0)
-    that.connectTo(unionAllOperator, 1)
-    DataQuantaHackit.wrap[Out](unionAllOperator)
+//    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+//    val unionAllOperator = new UnionAllOperator(dataSetType[Out])
+//    this.connectTo(unionAllOperator, 0)
+//    that.connectTo(unionAllOperator, 1)
+//    DataQuantaHackit.wrap[Out](unionAllOperator)
+    null
   }
 
   /**
@@ -328,11 +373,12 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[IntersectOperator]]'s output
    */
   override def intersect(that: DataQuanta[Out]): DataQuantaHackit[Out] =  {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val intersectOperator = new IntersectOperator(dataSetType[Out])
-    this.connectTo(intersectOperator, 0)
-    that.connectTo(intersectOperator, 1)
-    DataQuantaHackit.wrap[Out](intersectOperator)
+//    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+//    val intersectOperator = new IntersectOperator(dataSetType[Out])
+//    this.connectTo(intersectOperator, 0)
+//    that.connectTo(intersectOperator, 1)
+//    DataQuantaHackit.wrap[Out](intersectOperator)
+    null
   }
 
   /**
@@ -342,11 +388,12 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[CartesianOperator]]'s output
    */
   override def cartesian[ThatOut: ClassTag](that: DataQuanta[ThatOut]): DataQuantaHackit[WayangTuple2[Out, ThatOut]] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val cartesianOperator = new CartesianOperator(dataSetType[Out], dataSetType[ThatOut])
-    this.connectTo(cartesianOperator, 0)
-    that.connectTo(cartesianOperator, 1)
-    DataQuantaHackit.wrap[WayangTuple2[Out, ThatOut]](cartesianOperator)
+//    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+//    val cartesianOperator = new CartesianOperator(dataSetType[Out], dataSetType[ThatOut])
+//    this.connectTo(cartesianOperator, 0)
+//    that.connectTo(cartesianOperator, 1)
+//    DataQuantaHackit.wrap[WayangTuple2[Out, ThatOut]](cartesianOperator)
+    null
   }
 
   /**
@@ -355,9 +402,10 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[ZipWithIdOperator]]'s output
    */
   override def zipWithId: DataQuantaHackit[WayangTuple2[lang.Long, Out]] = {
-    val zipWithIdOperator = new ZipWithIdOperator(dataSetType[Out])
-    this.connectTo(zipWithIdOperator, 0)
-    DataQuantaHackit.wrap[WayangTuple2[lang.Long, Out]](zipWithIdOperator)
+//    val zipWithIdOperator = new ZipWithIdOperator(dataSetType[Out])
+//    this.connectTo(zipWithIdOperator, 0)
+//    DataQuantaHackit.wrap[WayangTuple2[lang.Long, Out]](zipWithIdOperator)
+    null
   }
 
   /**
@@ -366,9 +414,10 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[DistinctOperator]]'s output
    */
   override def distinct: DataQuantaHackit[Out] = {
-    val distinctOperator = new DistinctOperator(dataSetType[Out])
-    this.connectTo(distinctOperator, 0)
-    DataQuantaHackit.wrap[Out](distinctOperator)
+//    val distinctOperator = new DistinctOperator(dataSetType[Out])
+//    this.connectTo(distinctOperator, 0)
+//    DataQuantaHackit.wrap[Out](distinctOperator)
+    null
   }
 
   /**
@@ -377,9 +426,10 @@ class DataQuantaHackit[Out: ClassTag]
    * @return a new instance representing the [[CountOperator]]'s output
    */
   override def count: DataQuantaHackit[lang.Long] = {
-    val countOperator = new CountOperator(dataSetType[Out])
-    this.connectTo(countOperator, 0)
-    DataQuantaHackit.wrap[lang.Long](countOperator)
+//    val countOperator = new CountOperator(dataSetType[Out])
+//    this.connectTo(countOperator, 0)
+//    DataQuantaHackit.wrap[lang.Long](countOperator)
+    null
   }
 }
 
@@ -392,4 +442,8 @@ object DataQuantaHackit extends DataQuantaCreator{
   def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuantaHackit[_] =
     new DataQuantaHackit(output.getOwner.asInstanceOf[ElementaryOperator], output.getIndex)(ClassTag(output.getType.getDataUnitType.getTypeClass), planBuilder)
 
+  implicit def wrapDataQuanta[T:ClassTag](dataQuanta: DataQuanta[T]): DataQuantaHackit[T] = {
+    new DataQuantaHackit[T](dataQuanta.operator, dataQuanta.outputIndex)(ClassTag(dataQuanta.output.getType.getDataUnitType.getTypeClass), dataQuanta.planBuilder);
+  }
+
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/Hackit.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/Hackit.scala
new file mode 100644
index 0000000..b9499fb
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/main/scala/org/apache/wayang/plugin/hackit/api/Hackit.scala
@@ -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.wayang.plugin.hackit.api
+
+import org.apache.wayang.api.PlanBuilder
+import org.apache.wayang.api.dataquanta.DataQuanta
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
+import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger
+import org.apache.wayang.plugin.hackit.core.tagger.wrapper.FunctionWrapperHackit
+import org.apache.wayang.plugin.hackit.core.tags.HackitTag
+import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple
+import org.apache.wayang.api.toSerializableFunction
+
+import scala.language.implicitConversions
+import scala.reflect.{ClassTag, classTag}
+
+class Hackit[Key: ClassTag, Type:ClassTag](implicit var planBuilder: PlanBuilder) {
+
+  var dataQuanta: DataQuanta[HackitTuple[Key, Type]] = null
+
+  var tagger : HackitTagger = null;
+
+  def addTag(hackitTag: HackitTag) = {
+    println("you are adding a tag")
+    this
+  }
+
+  def underHackit(dataQuanta: DataQuanta[Type]): Hackit[Key, Type] = {
+    val hackit = new Hackit[Key, Type]()
+    hackit.dataQuanta = dataQuanta.map[HackitTuple[Key, Type]](
+      element => {
+        new HackitTuple[Key, Type](element)
+      }
+    )
+    hackit
+  }
+
+  def toDataQuanta(): DataQuanta[Type] = {
+    this.dataQuanta.map(tuple => tuple.getValue)
+  }
+
+  def map[TypeOut: ClassTag](udf: Type => TypeOut, udfLoad: LoadProfileEstimator = null): Hackit[Key, TypeOut] = {
+    val hackit = new Hackit[Key, TypeOut]()
+    val wrapper = new FunctionWrapperHackit[Key, Type, TypeOut](toSerializableFunction(udf))
+    hackit.dataQuanta = dataQuanta.map[HackitTuple[Key, TypeOut]](element => wrapper.apply(element), udfLoad)
+    hackit
+  }
+
+}
+
+
+object Hackit {
+
+  //TODO: replace the object with a parameter
+  implicit def underHackit[T: ClassTag](dataQuanta: DataQuanta[T]): Hackit[java.lang.Object, T] = {
+      return new Hackit[java.lang.Object, T]()(classTag[java.lang.Object], ClassTag(dataQuanta.output.getType.getDataUnitType.getTypeClass), dataQuanta.planBuilder)
+                  .underHackit(dataQuanta)
+  }
+
+}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala
index 59a0fd9..42867f3 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/plugin/hackit/api/ApiExtensionTest.scala
@@ -19,12 +19,17 @@
 
 package org.apache.wayang.plugin.hackit.api
 
-import org.apache.wayang.api.ApiTest
 import org.apache.wayang.api.dataquanta.DataQuantaFactory
-import org.junit.{BeforeClass, Test}
+import org.apache.wayang.api.createPlanBuilder
+import org.apache.wayang.plugin.hackit.api.Hackit.underHackit
+import org.apache.wayang.core.api.WayangContext
+import org.apache.wayang.java.Java
+import org.apache.wayang.spark.Spark
+import org.junit.{Assert, BeforeClass, Test}
 import org.junit.jupiter.api.{BeforeAll, BeforeEach}
 
-class ApiExtensionTest extends ApiTest {
+
+class ApiExtensionTest {
 
   @BeforeEach
   def setUp() ={
@@ -32,10 +37,35 @@ class ApiExtensionTest extends ApiTest {
   }
 
   @Test
-  override def testReadMapCollect(): Unit = {
-    DataQuantaFactory.setTemplate(DataQuantaHackit);
+  def testReadMapCollectHackit(): Unit = {
+
+    // Set up WayangContext.
+    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+
+    // Generate some test data.
+    val inputValues = (for (i <- 1 to 10) yield i).toArray
+
+    // Build and execute a Wayang plan.
+    var outputValues = wayang
+      .loadCollection(inputValues).withName("Load input values")
+      .addTag(null)
+      .map(a => a + 2)//.withName("Add 2")
+      .dataQuanta
+      .collect()
+
+    print(outputValues)
+
+    var lolo = wayang
+      .loadCollection(inputValues).withName("Load input values")
+      .addTag(null)
+      .map(a => a + 2)//.withName("Add 2")
+      .toDataQuanta()
+      .collect()
 
-    super.testReadMapCollect()
+    print(lolo)
+//    // Check the outcome.
+//    val expectedOutputValues = inputValues.map(_ + 2)
+//    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
   }
 
 }

[incubator-wayang] 05/15: [WAYANG-28] creation of module wayang-hackit-api

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit dd11607d0550e8e934dba581c87fd1328e73af4f
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Tue May 11 13:10:57 2021 -0400

    [WAYANG-28] creation of module wayang-hackit-api
---
 wayang-plugins/wayang-hackit/pom.xml               |  1 +
 .../wayang-hackit/wayang-hackit-api/pom.xml        | 41 ++++++++++++++++++++++
 2 files changed, 42 insertions(+)

diff --git a/wayang-plugins/wayang-hackit/pom.xml b/wayang-plugins/wayang-hackit/pom.xml
index b3e914f..6c07c29 100644
--- a/wayang-plugins/wayang-hackit/pom.xml
+++ b/wayang-plugins/wayang-hackit/pom.xml
@@ -40,6 +40,7 @@
         <module>wayang-hackit-core</module>
         <module>wayang-hackit-shipper</module>
         <module>wayang-hackit-sidecar</module>
+        <module>wayang-hackit-api</module>
     </modules>
 
     <properties>
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
new file mode 100644
index 0000000..0839a6b
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~   Licensed to the Apache Software Foundation (ASF) under one
+  ~   or more contributor license agreements.  See the NOTICE file
+  ~   distributed with this work for additional information
+  ~   regarding copyright ownership.  The ASF licenses this file
+  ~   to you under the Apache License, Version 2.0 (the
+  ~   "License"); you may not use this file except in compliance
+  ~   with the License.  You may obtain a copy of the License at
+  ~
+  ~       http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~   Unless required by applicable law or agreed to in writing,
+  ~   software distributed under the License is distributed on an
+  ~   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~   KIND, either express or implied.  See the License for the
+  ~   specific language governing permissions and limitations
+  ~   under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>wayang-hackit</artifactId>
+        <groupId>org.apache.wayang</groupId>
+        <version>0.6.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>wayang-hackit-api</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.wayang</groupId>
+            <artifactId>wayang-api-scala-java_2.11</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+
+</project>

[incubator-wayang] 09/15: [WAYANG-31] small correction to be extensible

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 99a666566d7a2882e0ce1333d7c86e5ac9ccabe7
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Fri May 14 08:54:56 2021 -0400

    [WAYANG-31] small correction to be extensible
---
 wayang-api/wayang-api-scala-java/pom.xml           | 25 ++++++++++++++--
 .../apache/wayang/api/dataquanta/DataQuanta.scala  |  2 +-
 .../wayang/api/dataquanta/DataQuantaCreator.scala  | 33 ++++++++++++++++++++++
 .../wayang/api/dataquanta/DataQuantaDefault.scala  |  6 ++--
 .../wayang/api/dataquanta/DataQuantaFactory.scala  | 21 +++++++++++++-
 5 files changed, 80 insertions(+), 7 deletions(-)

diff --git a/wayang-api/wayang-api-scala-java/pom.xml b/wayang-api/wayang-api-scala-java/pom.xml
index 1470c30..d385dae 100644
--- a/wayang-api/wayang-api-scala-java/pom.xml
+++ b/wayang-api/wayang-api-scala-java/pom.xml
@@ -91,7 +91,12 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <scope>provided</scope>
+            <version>${scala.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-reflect</artifactId>
+            <version>${scala.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
@@ -103,6 +108,11 @@
             <artifactId>hadoop-hdfs</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>0.9.12</version>
+        </dependency>
     </dependencies>
 
     <build>
@@ -114,8 +124,19 @@
                     <useSystemClassLoader>true</useSystemClassLoader>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.2</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 
-
 </project>
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
index 6e464e0..664079b 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
@@ -122,7 +122,7 @@ abstract class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outpu
     * @param operator   the [[Operator]] to connect to
     * @param inputIndex the input index of the [[Operator]]s [[InputSlot]]
     */
-  private[api] def connectTo(operator: Operator, inputIndex: Int): Unit =
+  def connectTo(operator: Operator, inputIndex: Int): Unit =
     this.operator.connectTo(outputIndex, operator, inputIndex)
 
 
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaCreator.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaCreator.scala
new file mode 100644
index 0000000..6388cc9
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaCreator.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.wayang.api.dataquanta
+
+import org.apache.wayang.api.PlanBuilder
+import org.apache.wayang.core.plan.wayangplan.{ElementaryOperator, OutputSlot}
+
+import scala.reflect.ClassTag
+
+abstract class DataQuantaCreator {
+
+  def wrap[T:ClassTag](operator: ElementaryOperator, outputIndex: Int = 0)(implicit planBuilder: PlanBuilder): DataQuanta[T]
+
+  def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuanta[_]
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala
index 1f79ae7..aa9c7d0 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala
@@ -19,7 +19,7 @@
 
 package org.apache.wayang.api.dataquanta
 
-import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, groupedDataSetType, groupedDataUnitType, toConsumer, toSerializableBinaryOperator, toSerializableFlatteningFunction, toSerializableFunction, toSerializablePartitionFunction, toSerializablePredicate}
+import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, groupedDataSetType, groupedDataUnitType}
 import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
 import org.apache.wayang.basic.function.ProjectionDescriptor
 import org.apache.wayang.basic.operators._
@@ -31,7 +31,7 @@ import org.apache.wayang.core.plan.wayangplan._
 
 import java.lang
 import java.lang.{Iterable => JavaIterable}
-import java.util.function.{Consumer, IntUnaryOperator, Function => JavaFunction}
+import java.util.function.{IntUnaryOperator}
 import scala.reflect._
 
 /**
@@ -396,7 +396,7 @@ class DataQuantaDefault[Out: ClassTag]
   }
 }
 
-object DataQuantaDefault {
+object DataQuantaDefault extends DataQuantaCreator {
 
   def wrap[T:ClassTag](operator: ElementaryOperator, outputIndex: Int = 0)(implicit planBuilder: PlanBuilder): DataQuantaDefault[T] = {
     new DataQuantaDefault[T](operator, outputIndex)
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala
index aad0e42..ef8fe0d 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala
@@ -21,7 +21,11 @@ package org.apache.wayang.api.dataquanta
 
 import org.apache.wayang.api.PlanBuilder
 import org.apache.wayang.core.plan.wayangplan.ElementaryOperator
+import org.reflections.Reflections
+import org.reflections.scanners.{FieldAnnotationsScanner, MethodParameterScanner, ResourcesScanner, SubTypesScanner}
+import org.reflections.util.{ClasspathHelper, ConfigurationBuilder}
 
+import scala.collection.immutable.List
 import scala.reflect.ClassTag
 
 /**
@@ -32,6 +36,21 @@ import scala.reflect.ClassTag
 object DataQuantaFactory {
 
   /**
+   * template is the instance of [[DataQuantaCreator]] that will be use in the creation of [[DataQuanta]] instance
+   */
+  var template: DataQuantaCreator = DataQuantaDefault
+
+  /**
+   * set the [[DataQuantaCreator]]
+   *
+   * @param dataQuantaCreator it will be use as creator when the [[DataQuantaFactory.build()]] is called
+   */
+  def setTemplate(dataQuantaCreator: DataQuantaCreator) = {
+    this.template = dataQuantaCreator
+
+  }
+
+  /**
    * Given the configuration loaded the [[DataQuantaFactory.build()]] the right extender, if not configuration is
    * provided the [[DataQuantaFactory]] will create a [[DataQuantaDefault]] instance
    *
@@ -43,7 +62,7 @@ object DataQuantaFactory {
    */
   def build[T:ClassTag](operator: ElementaryOperator, outputIndex: Int = 0)(implicit planBuilder: PlanBuilder): DataQuanta[T] = {
     //TODO validate if the correct way
-    DataQuantaDefault.wrap[T](operator, outputIndex)
+    this.template.wrap[T](operator, outputIndex)
   }
 
 }

[incubator-wayang] 13/15: [WAYANG-28] use base java function on the tagger template

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 5e26844dc14959ffe9c57bc5d8955ab8d82475fc
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Mon May 17 12:48:34 2021 -0400

    [WAYANG-28] use base java function on the tagger template
---
 .../core/tagger/wrapper/FlatmapWrapperHackit.java  | 19 +++++-----
 .../core/tagger/wrapper/FunctionWrapperHackit.java | 19 +++++-----
 .../tagger/wrapper/PredicateWrapperHackit.java     | 23 ++++++------
 .../template/TaggerWrapperFlatMapTemplate.java     | 41 ----------------------
 .../template/TaggerWrapperFunctionTemplate.java    | 36 -------------------
 5 files changed, 31 insertions(+), 107 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
index 5190f26..5d020a2 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
@@ -18,13 +18,13 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.TaggerWrapperFlatMapTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
 import java.util.Iterator;
+import java.util.function.Function;
 
 /**
- * FlatmapWrapperHackit is an implementation of {@link TaggerWrapperFlatMapTemplate} where Hackit manage the logic
+ * FlatmapWrapperHackit is an implementation of {@link HackitTagger} where Hackit manage the logic
  * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
  * original function
  *
@@ -34,27 +34,26 @@ import java.util.Iterator;
  */
 public class FlatmapWrapperHackit<IDType, I, O>
         extends HackitTagger
-        implements TaggerWrapperFlatMapTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
+        implements Function<HackitTuple<IDType, I>, Iterator<HackitTuple<IDType, O>>> {
 
     /**
      * Original function that will transform the data
      */
-    private TaggerWrapperFlatMapTemplate<I, O> function;
+    private Function<I, Iterator<O>> function;
 
     /**
      * Default Construct
      *
      * @param function is the function that will be Wrapped by the {@link FlatmapWrapperHackit}
      */
-    public FlatmapWrapperHackit(TaggerWrapperFlatMapTemplate<I, O> function ) {
+    public FlatmapWrapperHackit(Function<I, Iterator<O>> function ) {
         this.function = function;
     }
 
-
     @Override
-    public Iterator<HackitTuple<IDType, O>> execute(HackitTuple<IDType, I> kiHackItTuple) {
-        this.preTaggingTuple(kiHackItTuple);
-        Iterator<O> result = this.function.execute(kiHackItTuple.getValue());
-        return this.postTaggingTuple(kiHackItTuple, result);
+    public Iterator<HackitTuple<IDType, O>> apply(HackitTuple<IDType, I> idTypeIHackitTuple) {
+        this.preTaggingTuple(idTypeIHackitTuple);
+        Iterator<O> result = this.function.apply(idTypeIHackitTuple.getValue());
+        return this.postTaggingTuple(idTypeIHackitTuple, result);
     }
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
index 982b838..4c97b28 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
@@ -18,11 +18,12 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.TaggerWrapperFunctionTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
+import java.util.function.Function;
+
 /**
- * FunctionWrapperHackit is an implementation of {@link TaggerWrapperFunctionTemplate} where Hackit manage the logic
+ * FunctionWrapperHackit is an implementation of {@link HackitTagger} where Hackit manage the logic
  * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
  * original function
  *
@@ -32,26 +33,26 @@ import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
  */
 public class FunctionWrapperHackit<IDType, I, O>
         extends HackitTagger
-        implements TaggerWrapperFunctionTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
+        implements Function<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
 
     /**
      * Original function that will transform the data
      */
-    private TaggerWrapperFunctionTemplate<I, O> function;
+    private Function<I, O> function;
 
     /**
      * Default Construct
      *
      * @param function is the function that will be Wrapped by the {@link FunctionWrapperHackit}
      */
-    public FunctionWrapperHackit(TaggerWrapperFunctionTemplate<I, O> function) {
+    public FunctionWrapperHackit(Function<I, O> function) {
         this.function = function;
     }
 
     @Override
-    public HackitTuple<IDType, O> execute(HackitTuple<IDType, I> v1) {
-        this.preTaggingTuple(v1);
-        O result = this.function.execute(v1.getValue());
-        return this.postTaggingTuple(v1, result);
+    public HackitTuple<IDType, O> apply(HackitTuple<IDType, I> idTypeIHackitTuple) {
+        this.preTaggingTuple(idTypeIHackitTuple);
+        O result = this.function.apply(idTypeIHackitTuple.getValue());
+        return this.postTaggingTuple(idTypeIHackitTuple, result);
     }
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
index f9d47cd..945cbfb 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
@@ -18,41 +18,42 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
-import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.TaggerWrapperFunctionTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
+import java.util.function.Predicate;
+
 /**
- * PredicateWrapperHackit is an implementation of {@link TaggerWrapperFunctionTemplate} where Hackit manage the logic
+ * PredicateWrapperHackit is an implementation of {@link HackitTagger} where Hackit manage the logic
  * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
- * original function. The original {@link TaggerWrapperFunctionTemplate} it an predicate function because return a
- * {@link Boolean}
+ * original function. The original {@link Predicate} function because return a {@link Boolean}
  *
  * @param <IDType> Type of {@link org.apache.wayang.plugin.hackit.core.tuple.header.Header} key of the {@link HackitTuple}
  * @param <I> Input Type of the original Tuple to be evaluated
  */
 public class PredicateWrapperHackit<IDType, I>
         extends HackitTagger
-        implements TaggerWrapperFunctionTemplate<HackitTuple<IDType, I>, Boolean> {
+        implements Predicate<HackitTuple<IDType, I>> {
 
     /**
      * Original predicate that will evaluate the data to give a True or False value
      */
-    private TaggerWrapperFunctionTemplate<I, Boolean> function;
+    private Predicate<I> function;
 
     /**
      * Default Construct
      *
      * @param function is the predicate that will be Wrapped by the {@link PredicateWrapperHackit}
      */
-    public PredicateWrapperHackit(TaggerWrapperFunctionTemplate<I, Boolean> function) {
+    public PredicateWrapperHackit(Predicate<I> function) {
         this.function = function;
     }
 
+
     @Override
-    public Boolean execute(HackitTuple<IDType, I> v1) {
-        this.preTaggingTuple(v1);
-        Boolean result = this.function.execute(v1.getValue());
-        this.postTaggingTuple(v1);
+    public boolean test(HackitTuple<IDType, I> idTypeIHackitTuple) {
+        this.preTaggingTuple(idTypeIHackitTuple);
+        Boolean result = this.function.test(idTypeIHackitTuple.getValue());
+        this.postTaggingTuple(idTypeIHackitTuple);
         return result;
     }
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFlatMapTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFlatMapTemplate.java
deleted file mode 100644
index bb05c3a..0000000
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFlatMapTemplate.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.wayang.plugin.hackit.core.tagger.wrapper.template;
-
-import java.util.Iterator;
-
-/**
- * TaggerWrapperFlatMapTemplate is the template that provide the abstraction to work with Flatmap operations and also
- * allows to wrap some function made by the user.
- *
- * TaggerWrapperFlatMapTemplate generate as output a {@link Iterator} this could be an extension of {@link org.apache.wayang.plugin.hackit.core.iterator.HackitIterator}
- *
- * @param <I> Input type of the original Function
- * @param <O> Output type of the original function
- */
-public interface TaggerWrapperFlatMapTemplate<I, O> extends TaggerWrapperFunctionTemplate<I, Iterator<O>> {
-
-    /**
-     * Execute the logic over one element and generate as output a {@link Iterator}
-     *
-     * @param input element to transform
-     * @return {@link Iterator} that contains the output's
-     */
-    Iterator<O> execute(I input);
-}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFunctionTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFunctionTemplate.java
deleted file mode 100644
index 1033d4f..0000000
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/TaggerWrapperFunctionTemplate.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.wayang.plugin.hackit.core.tagger.wrapper.template;
-
-/**
- * TaggerWrapperFunctionTemplate is the template that provide the abstraction to work with Transformation operations and also
- * allows to wrap some function made by the user.
- *
- * @param <I> Input type of the original Function
- * @param <O> Output type of the original function
- */
-public interface TaggerWrapperFunctionTemplate<I, O> {
-
-    /**
-     * Execute the logic over one element and generate as output <code>T</code>
-     *
-     * @param input element to transform
-     * @return <code>O</code> that is the transformation of the <code>input</code>
-     */
-    public O execute(I input);
-}

[incubator-wayang] 02/15: [WAYANG-28] Add javadoc to tagger package

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 0890650638428cac7a1a37392855a5cf796a27b9
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Wed May 5 00:39:03 2021 -0400

    [WAYANG-28] Add javadoc to tagger package
---
 .../plugin/hackit/core/tagger/HackitTagger.java    | 76 ++++++++++++++++++++++
 .../plugin/hackit/core/tagger/TaggerFunction.java  | 19 +++++-
 .../core/tagger/wrapper/FlatmapWrapperHackit.java  | 21 +++++-
 .../core/tagger/wrapper/FunctionWrapperHackit.java | 22 ++++++-
 .../tagger/wrapper/PredicateWrapperHackit.java     | 21 +++++-
 .../tagger/wrapper/template/FlatMapTemplate.java   | 15 +++++
 .../tagger/wrapper/template/FunctionTemplate.java  | 13 ++++
 7 files changed, 180 insertions(+), 7 deletions(-)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
index 87a01c9..48002de 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
@@ -27,13 +27,38 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+/**
+ * HackitTagger is class where is allocated all the logic that need to be perform during the
+ * tagging step in Hackit, this logic have and pre and post processing and they are acting like
+ * template that follow same behaivor in every tagger
+ */
 public class HackitTagger implements Serializable {
 
+    /**
+     * {@link List} of {@link HackitTag} that are added previous of the execution of the
+     * original function
+     */
+    //TODO: It may change by a set
     protected List<HackitTag> pre_tags;
+
+    /**
+     * {@link List} of {@link HackitTag} that are added after of the execution of the
+     * original function
+     */
+    //TODO: It may change by a set
     protected List<HackitTag> post_tags;
 
+    /**
+     * Default Construct
+     */
     public HackitTagger(){}
 
+    /**
+     * Add a {@link HackitTag} to {@link List} of <code>pre_tags</code>
+     *
+     * @param tag is a {@link HackitTag} added to change the future behavior
+     * @return {@link HackitTagger} as it self reference
+     */
     public HackitTagger addPreTag(HackitTag tag){
         if(this.pre_tags == null){
             this.pre_tags = new ArrayList<>();
@@ -42,6 +67,12 @@ public class HackitTagger implements Serializable {
         return this;
     }
 
+    /**
+     * Add a {@link HackitTag} to {@link List} of <code>post_tags</code>
+     *
+     * @param tag is a {@link HackitTag} added to change the future behavior
+     * @return {@link HackitTagger} as it self reference
+     */
     public HackitTagger addPostTag(HackitTag tag){
         if(this.post_tags == null){
             this.post_tags = new ArrayList<>();
@@ -50,20 +81,51 @@ public class HackitTagger implements Serializable {
         return this;
     }
 
+    /**
+     * add to the {@link HackitTuple} all the {@link HackitTag}'s add pre-tagging phase are available at that moment
+     *
+     * @param tuple is a {@link HackitTuple} where that {@link HackitTag} will be added
+     */
     public void preTaggingTuple(HackitTuple tuple){
         if(this.pre_tags != null)
             taggingTuple(tuple, this.pre_tags);
     }
 
+    /**
+     * add to the {@link HackitTuple} all the {@link HackitTag}'s add post-tagging phase are available at that moment
+     *
+     * @param tuple is a {@link HackitTuple} where that {@link HackitTag} will be added
+     */
     public void postTaggingTuple(HackitTuple tuple){
         if(this.post_tags != null)
             taggingTuple(tuple, this.post_tags);
     }
 
+    /**
+     * add all the {@link HackitTag}'s available on the {@link List} to the {@link HackitTuple}
+     *
+     * @param tuple is {@link HackitTuple} where the tags will be added
+     * @param tags {@link List} of {@link HackitTag}'s that will add to {@link HackitTuple}
+     */
     public void taggingTuple(HackitTuple tuple, List<HackitTag> tags){
+        //TODO: change this code for an efficient one
         tags.stream().forEach(tag -> tuple.addTag(tag.getInstance()));
     }
 
+    /**
+     * It take the original {@link HackitTuple} and extract the {@link Header} and start creating the children
+     * from that {@link Header} this enable to follow the lineage of the after a several elements come out from
+     * one {@link HackitTuple}. This generation is possible by inserting a new step at the iterator using {@link HackitIterator}
+     * that allow append a new instruction in the process that will be perform on the original {@link Iterator}
+     *
+     * @param origin Original {@link HackitTuple} that it was transformed
+     * @param result is the transformation output inside of an {@link Iterator}
+     * @param <K> type of the identifier of {@link HackitTuple}
+     * @param <I> type of the original element inside of {@link HackitTuple} that it was transformed
+     * @param <O> type of the output in the transformation
+     *
+     * @return {@link Iterator} that is wrapper of the original with the add instruction using {@link HackitIterator}
+     */
     public <K, I, O> Iterator<HackitTuple<K,O>> postTaggingTuple(HackitTuple<K, I> origin, Iterator<O>result){
         Header<K> header = origin.getHeader();
         Iterator<HackitTuple<K, O>> iter_result = new HackitIterator<K, O>(
@@ -80,6 +142,20 @@ public class HackitTagger implements Serializable {
         return iter_result;
     }
 
+    /**
+     * It take the original {@link HackitTuple} and extract the {@link Header} to be enable to follow the lineage of the
+     * same value transformed, then is generate a new {@link HackitTuple} with the <code>result</code> as the wrapped
+     * element, it also add the new {@link HackitTag}'s to the {@link Header}
+     *
+     *
+     * @param origin Original {@link HackitTuple} that it was transformed
+     * @param result is the transformation output
+     * @param <K> type of the identifier of {@link HackitTuple}
+     * @param <I> type of the original element inside of {@link HackitTuple} that it was transformed
+     * @param <O> type of the output in the transformation
+     *
+     * @return {@link HackitTuple} with the new {@link HackitTag}
+     */
     public <K, I, O> HackitTuple<K, O> postTaggingTuple(HackitTuple<K, I> origin, O result){
         HackitTuple<K, O> hackItTuple_result = new HackitTuple<K, O>(origin.getHeader(), result);
         this.postTaggingTuple(hackItTuple_result);
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/TaggerFunction.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/TaggerFunction.java
index 186b327..eb26a4b 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/TaggerFunction.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/TaggerFunction.java
@@ -17,9 +17,24 @@
  */
 package org.apache.wayang.plugin.hackit.core.tagger;
 
+/**
+ * TaggerFunction is the template for the function that will be use inside of {@link HackitTagger}
+ *
+ * @param <T> output type of the execution
+ */
 public interface TaggerFunction<T> {
-    public T execute();
+    /**
+     * It execute the function of tagger, and generate and output
+     *
+     * @return result of the execution
+     */
+    T execute();
 
-    public String getName();
+    /**
+     * Get the name of the function, this is use a identifier
+     *
+     * @return {@link String} that contains the name of the {@link TaggerFunction}
+     */
+    String getName();
 
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
index a57cd23..9100c99 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FlatmapWrapperHackit.java
@@ -23,15 +23,30 @@ import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
 import java.util.Iterator;
 
+/**
+ * FlatmapWrapperHackit is an implementation of {@link FlatMapTemplate} where Hackit manage the logic
+ * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
+ * original function
+ *
+ * @param <IDType> Type of {@link org.apache.wayang.plugin.hackit.core.tuple.header.Header} key of the {@link HackitTuple}
+ * @param <I> Input Type of the original Tuple
+ * @param <O> Output Type after the perform the Function
+ */
 public class FlatmapWrapperHackit<IDType, I, O>
         extends HackitTagger
         implements FlatMapTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
 
+    /**
+     * Original function that will transform the data
+     */
     private FlatMapTemplate<I, O> function;
 
-    public FlatmapWrapperHackit(
-            FlatMapTemplate<I, O> function
-    ) {
+    /**
+     * Default Construct
+     *
+     * @param function is the function that will be Wrapped by the {@link FlatmapWrapperHackit}
+     */
+    public FlatmapWrapperHackit( FlatMapTemplate<I, O> function ) {
         this.function = function;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
index c244eb0..99e4b37 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/FunctionWrapperHackit.java
@@ -18,13 +18,33 @@
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper;
 
 import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
+import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FlatMapTemplate;
 import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FunctionTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
-public class FunctionWrapperHackit <IDType, I, O> extends HackitTagger implements FunctionTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
+/**
+ * FunctionWrapperHackit is an implementation of {@link FunctionTemplate} where Hackit manage the logic
+ * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
+ * original function
+ *
+ * @param <IDType> Type of {@link org.apache.wayang.plugin.hackit.core.tuple.header.Header} key of the {@link HackitTuple}
+ * @param <I> Input Type of the original Tuple
+ * @param <O> Output Type after the perform the Function
+ */
+public class FunctionWrapperHackit <IDType, I, O>
+        extends HackitTagger
+        implements FunctionTemplate<HackitTuple<IDType, I>, HackitTuple<IDType, O>> {
 
+    /**
+     * Original function that will transform the data
+     */
     private FunctionTemplate<I, O> function;
 
+    /**
+     * Default Construct
+     *
+     * @param function is the function that will be Wrapped by the {@link FunctionWrapperHackit}
+     */
     public FunctionWrapperHackit(FunctionTemplate<I, O> function) {
         this.function = function;
     }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
index c2636e2..7249fdf 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/PredicateWrapperHackit.java
@@ -21,10 +21,29 @@ import org.apache.wayang.plugin.hackit.core.tagger.HackitTagger;
 import org.apache.wayang.plugin.hackit.core.tagger.wrapper.template.FunctionTemplate;
 import org.apache.wayang.plugin.hackit.core.tuple.HackitTuple;
 
-public class PredicateWrapperHackit<IDType, I> extends HackitTagger implements FunctionTemplate<HackitTuple<IDType, I>, Boolean> {
+/**
+ * PredicateWrapperHackit is an implementation of {@link FunctionTemplate} where Hackit manage the logic
+ * before and after of tagging process, also it perform the unwrap of the tuple to be handle by the
+ * original function. The original {@link FunctionTemplate} it an predicate function because return a
+ * {@link Boolean}
+ *
+ * @param <IDType> Type of {@link org.apache.wayang.plugin.hackit.core.tuple.header.Header} key of the {@link HackitTuple}
+ * @param <I> Input Type of the original Tuple to be evaluated
+ */
+public class PredicateWrapperHackit<IDType, I>
+        extends HackitTagger
+        implements FunctionTemplate<HackitTuple<IDType, I>, Boolean> {
 
+    /**
+     * Original predicate that will evaluate the data to give a True or False value
+     */
     private FunctionTemplate<I, Boolean> function;
 
+    /**
+     * Default Construct
+     *
+     * @param function is the predicate that will be Wrapped by the {@link PredicateWrapperHackit}
+     */
     public PredicateWrapperHackit(FunctionTemplate<I, Boolean> function) {
         this.function = function;
     }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.java
index abe6471..6c68f0c 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FlatMapTemplate.java
@@ -19,7 +19,22 @@ package org.apache.wayang.plugin.hackit.core.tagger.wrapper.template;
 
 import java.util.Iterator;
 
+/**
+ * FlatMapTemplate is the template that provide the abstraction to work with Flatmap operations and also
+ * allows to wrap some function made by the user.
+ *
+ * FlatMapTemplate generate as output a {@link Iterator} this could be an extension of {@link org.apache.wayang.plugin.hackit.core.iterator.HackitIterator}
+ *
+ * @param <I> Input type of the original Function
+ * @param <O> Output type of the original function
+ */
 public interface FlatMapTemplate<I, O> {
 
+    /**
+     * Execute the logic over one element and generate as output a {@link Iterator}
+     *
+     * @param input element to transform
+     * @return {@link Iterator} that contains the output's
+     */
     public Iterator<O> execute(I input);
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java
index 9e5cb65..e33ad77 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/wrapper/template/FunctionTemplate.java
@@ -17,7 +17,20 @@
  */
 package org.apache.wayang.plugin.hackit.core.tagger.wrapper.template;
 
+/**
+ * FunctionTemplate is the template that provide the abstraction to work with Transformation operations and also
+ * allows to wrap some function made by the user.
+ *
+ * @param <I> Input type of the original Function
+ * @param <O> Output type of the original function
+ */
 public interface FunctionTemplate<I, O> {
 
+    /**
+     * Execute the logic over one element and generate as output <code>T</code>
+     *
+     * @param input element to transform
+     * @return <code>O</code> that is the transformation of the <code>input</code>
+     */
     public O execute(I input);
 }

[incubator-wayang] 07/15: [WAYANG-31] structure the code on different way

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit d39ab4f63c1967fc856f64c6aca07eec3fb07807
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Thu May 13 12:10:46 2021 -0400

    [WAYANG-31] structure the code on different way
---
 lolo                                               | 121 ++++
 .../org/apache/wayang/api/DataQuantaBuilder.scala  | 440 -------------
 .../org/apache/wayang/api/JavaPlanBuilder.scala    |   3 +-
 .../scala/org/apache/wayang/api/PlanBuilder.scala  |   1 +
 .../org/apache/wayang/api/RecordDataQuanta.scala   |   1 +
 .../wayang/api/RecordDataQuantaBuilder.scala       |   3 +-
 .../wayang/api/{ => dataquanta}/DataQuanta.scala   |  98 +--
 .../wayang/api/dataquanta/DataQuantaBuilder.scala  | 441 +++++++++++++
 .../wayang/api/dataquanta/JoinedDataQuanta.scala   |  56 ++
 .../wayang/api/dataquanta/KeyedDataQuanta.scala    |  53 ++
 .../builder}/BasicDataQuantaBuilder.scala          |   5 +-
 .../builder}/CartesianDataQuantaBuilder.scala      |   5 +-
 .../builder}/CoGroupDataQuantaBuilder.scala        |   5 +-
 .../builder}/CountDataQuantaBuilder.scala          |   5 +-
 .../builder}/CustomOperatorDataQuantaBuilder.scala |   5 +-
 .../builder}/DistinctDataQuantaBuilder.scala       |   5 +-
 .../builder}/DoWhileDataQuantaBuilder.scala        |   5 +-
 .../builder}/FakeDataQuantaBuilder.scala           |   5 +-
 .../builder}/FilterDataQuantaBuilder.scala         |   5 +-
 .../builder}/FlatMapDataQuantaBuilder.scala        |   5 +-
 .../builder}/GlobalGroupDataQuantaBuilder.scala    |   5 +-
 .../builder}/GlobalReduceDataQuantaBuilder.scala   |   5 +-
 .../builder}/GroupByDataQuantaBuilder.scala        |   5 +-
 .../builder}/IntersectDataQuantaBuilder.scala      |   5 +-
 .../builder}/JoinDataQuantaBuilder.scala           |   5 +-
 .../builder}/KeyedDataQuantaBuilder.scala          |   5 +-
 .../builder}/LoadCollectionDataQuantaBuilder.scala |   5 +-
 .../builder}/MapDataQuantaBuilder.scala            |   5 +-
 .../builder}/MapPartitionsDataQuantaBuilder.scala  |   5 +-
 .../builder}/ProjectionDataQuantaBuilder.scala     |   5 +-
 .../builder}/ReduceByDataQuantaBuilder.scala       |   5 +-
 .../builder}/RepeatDataQuantaBuilder.scala         |   5 +-
 .../builder}/SampleDataQuantaBuilder.scala         |   5 +-
 .../builder}/SortDataQuantaBuilder.scala           |   5 +-
 .../builder}/UnarySourceDataQuantaBuilder.scala    |   5 +-
 .../builder}/UnionDataQuantaBuilder.scala          |   5 +-
 .../builder}/ZipWithIdDataQuantaBuilder.scala      |   5 +-
 .../apache/wayang/api/graph/EdgeDataQuanta.scala   |   1 +
 .../wayang/api/graph/EdgeDataQuantaBuilder.scala   |   5 +-
 .../org/apache/wayang/api/graph/package.scala      |   1 +
 .../main/scala/org/apache/wayang/api/package.scala |   3 +-
 .../wayang/api/util/DataQuantaBuilderCache.scala   |   4 +-
 .../api/util/DataQuantaBuilderDecorator.scala      |   3 +-
 .../java/org/apache/wayang/api/JavaApiTest.java    |   5 +-
 .../test/scala/org/apache/wayang/api/ApiTest.scala |   3 +-
 .../wayang-hackit/wayang-hackit-api/pom.xml        |  33 +
 .../java/org/apache/wayang/api/JavaApiTest.java    | 711 +++++++++++++++++++++
 .../test/scala/org/apache/wayang/api/ApiTest.scala | 575 +++++++++++++++++
 .../plugin/hackit/core/tagger/HackitTagger.java    |   1 +
 .../java/org/apache/wayang/tests/RegressionIT.java |   4 +-
 .../java/org/apache/wayang/tests/WayangPlans.java  |   2 +-
 51 files changed, 2117 insertions(+), 586 deletions(-)

diff --git a/lolo b/lolo
new file mode 100644
index 0000000..11a7c99
--- /dev/null
+++ b/lolo
@@ -0,0 +1,121 @@
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/bin/java -ea -Didea.test.cyclic.buffer.size=1048576 -javaagent
+/Users/bertty/Library/Application Support/JetBrains/Toolbox/apps/IDEA-U/ch-0/211.7142.45/IntelliJ IDEA.app/Contents/lib/idea_rt.jar=61718
+/Users/bertty/Library/Application Support/JetBrains/Toolbox/apps/IDEA-U/ch-0/211.7142.45/IntelliJ IDEA.app/Contents/bin -Dfile.encoding=UTF-8 -classpath /Users/bertty/Library/Application Support/JetBrains/Toolbox/apps/IDEA-U/ch-0/211.7142.45/IntelliJ IDEA.app/Contents/lib/idea_rt.jar
+/Users/bertty/.m2/repository/org/junit/platform/junit-platform-launcher/1.6.1/junit-platform-launcher-1.6.1.jar
+/Users/bertty/Library/Application Support/JetBrains/Toolbox/apps/IDEA-U/ch-0/211.7142.45/IntelliJ IDEA.app/Contents/plugins/junit/lib/junit5-rt.jar
+/Users/bertty/Library/Application Support/JetBrains/Toolbox/apps/IDEA-U/ch-0/211.7142.45/IntelliJ IDEA.app/Contents/plugins/junit/lib/junit-rt.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/charsets.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/cldrdata.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/dnsns.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/jaccess.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/jfxrt.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/localedata.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/nashorn.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/sunec.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/sunjce_provider.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/sunpkcs11.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/ext/zipfs.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/jce.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/jfxswt.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/jsse.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/management-agent.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/resources.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/jre/lib/rt.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/ant-javafx.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/dt.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/javafx-mx.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/jconsole.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/packager.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/sa-jdi.jar
+/Users/bertty/Library/Java/JavaVirtualMachines/corretto-1.8.0_252/Contents/Home/lib/tools.jar
+/Users/bertty/IdeaProjects/incubator-wayang/wayang-plugins/wayang-hackit/wayang-hackit-api/target/test-classes
+/Users/bertty/IdeaProjects/incubator-wayang/wayang-plugins/wayang-hackit/wayang-hackit-api/target/classes
+/Users/bertty/IdeaProjects/incubator-wayang/wayang-api/wayang-api-scala-java/target/classes
+/Users/bertty/IdeaProjects/incubator-wayang/wayang-commons/wayang-core/target/classes
+/Users/bertty/.m2/repository/org/json/json/20160212/json-20160212.jar
+/Users/bertty/.m2/repository/org/yaml/snakeyaml/1.17/snakeyaml-1.17.jar
+/Users/bertty/.m2/repository/de/odysseus/juel/juel-api/2.2.7/juel-api-2.2.7.jar
+/Users/bertty/.m2/repository/de/odysseus/juel/juel-impl/2.2.7/juel-impl-2.2.7.jar
+/Users/bertty/.m2/repository/de/hpi/isg/profiledb-instrumentation/0.1.1/profiledb-instrumentation-0.1.1.jar
+/Users/bertty/.m2/repository/org/apache/commons/commons-lang3/3.3.2/commons-lang3-3.3.2.jar
+/Users/bertty/.m2/repository/org/apache/logging/log4j/log4j-api/2.14.0/log4j-api-2.14.0.jar
+/Users/bertty/.m2/repository/org/apache/logging/log4j/log4j-core/2.14.0/log4j-core-2.14.0.jar
+/Users/bertty/.m2/repository/net/sf/trove4j/trove4j/3.0.3/trove4j-3.0.3.jar
+/Users/bertty/IdeaProjects/incubator-wayang/wayang-commons/wayang-basic/target/classes
+/Users/bertty/.m2/repository/de/hpi/isg/profiledb-store/0.1.1/profiledb-store-0.1.1.jar
+/Users/bertty/IdeaProjects/incubator-wayang/wayang-platforms/wayang-java/target/classes
+/Users/bertty/.m2/repository/org/scala-lang/scala-library/2.11.12/scala-library-2.11.12.jar
+/Users/bertty/.m2/repository/org/apache/hadoop/hadoop-common/2.7.7/hadoop-common-2.7.7.jar
+/Users/bertty/.m2/repository/org/apache/hadoop/hadoop-annotations/2.7.7/hadoop-annotations-2.7.7.jar
+/Users/bertty/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar
+/Users/bertty/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar
+/Users/bertty/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar
+/Users/bertty/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar
+/Users/bertty/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar
+/Users/bertty/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar
+/Users/bertty/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar
+/Users/bertty/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar
+/Users/bertty/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar
+/Users/bertty/.m2/repository/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar
+/Users/bertty/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar
+/Users/bertty/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar
+/Users/bertty/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar
+/Users/bertty/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar
+/Users/bertty/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar
+/Users/bertty/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar
+/Users/bertty/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar
+/Users/bertty/.m2/repository/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar
+/Users/bertty/.m2/repository/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar
+/Users/bertty/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar
+/Users/bertty/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar
+/Users/bertty/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar
+/Users/bertty/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar
+/Users/bertty/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar
+/Users/bertty/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar
+/Users/bertty/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar
+/Users/bertty/.m2/repository/org/apache/hadoop/hadoop-auth/2.7.7/hadoop-auth-2.7.7.jar
+/Users/bertty/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar
+/Users/bertty/.m2/repository/org/apache/httpcomponents/httpcore/4.2.4/httpcore-4.2.4.jar
+/Users/bertty/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar
+/Users/bertty/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar
+/Users/bertty/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar
+/Users/bertty/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar
+/Users/bertty/.m2/repository/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar
+/Users/bertty/.m2/repository/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar
+/Users/bertty/.m2/repository/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar
+/Users/bertty/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar
+/Users/bertty/.m2/repository/org/apache/htrace/htrace-core/3.1.0-incubating/htrace-core-3.1.0-incubating.jar
+/Users/bertty/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar
+/Users/bertty/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar
+/Users/bertty/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar
+/Users/bertty/.m2/repository/org/apache/hadoop/hadoop-hdfs/2.7.7/hadoop-hdfs-2.7.7.jar
+/Users/bertty/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar
+/Users/bertty/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar
+/Users/bertty/.m2/repository/io/netty/netty-all/4.0.23.Final/netty-all-4.0.23.Final.jar
+/Users/bertty/.m2/repository/xerces/xercesImpl/2.9.1/xercesImpl-2.9.1.jar
+/Users/bertty/.m2/repository/xml-apis/xml-apis/1.3.04/xml-apis-1.3.04.jar
+/Users/bertty/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar
+/Users/bertty/.m2/repository/org/junit/jupiter/junit-jupiter/5.6.1/junit-jupiter-5.6.1.jar
+/Users/bertty/.m2/repository/org/junit/jupiter/junit-jupiter-api/5.6.1/junit-jupiter-api-5.6.1.jar
+/Users/bertty/.m2/repository/org/opentest4j/opentest4j/1.2.0/opentest4j-1.2.0.jar
+/Users/bertty/.m2/repository/org/junit/platform/junit-platform-commons/1.6.1/junit-platform-commons-1.6.1.jar
+/Users/bertty/.m2/repository/org/junit/jupiter/junit-jupiter-params/5.6.1/junit-jupiter-params-5.6.1.jar
+/Users/bertty/.m2/repository/org/junit/jupiter/junit-jupiter-engine/5.6.1/junit-jupiter-engine-5.6.1.jar
+/Users/bertty/.m2/repository/org/junit/vintage/junit-vintage-engine/5.6.1/junit-vintage-engine-5.6.1.jar
+/Users/bertty/.m2/repository/org/apiguardian/apiguardian-api/1.1.0/apiguardian-api-1.1.0.jar
+/Users/bertty/.m2/repository/org/junit/platform/junit-platform-engine/1.6.1/junit-platform-engine-1.6.1.jar
+/Users/bertty/.m2/repository/junit/junit/4.12/junit-4.12.jar
+/Users/bertty/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar
+/Users/bertty/.m2/repository/org/mockito/mockito-core/3.5.10/mockito-core-3.5.10.jar
+/Users/bertty/.m2/repository/net/bytebuddy/byte-buddy/1.10.13/byte-buddy-1.10.13.jar
+/Users/bertty/.m2/repository/net/bytebuddy/byte-buddy-agent/1.10.13/byte-buddy-agent-1.10.13.jar
+/Users/bertty/.m2/repository/org/objenesis/objenesis/3.1/objenesis-3.1.jar
+/Users/bertty/.m2/repository/org/mockito/mockito-junit-jupiter/3.5.10/mockito-junit-jupiter-3.5.10.jar
+/Users/bertty/.m2/repository/org/assertj/assertj-core/3.17.2/assertj-core-3.17.2.jar
+/Users/bertty/.m2/repository/org/antlr/antlr4/4.9.1/antlr4-4.9.1.jar
+/Users/bertty/.m2/repository/org/antlr/antlr4-runtime/4.9.1/antlr4-runtime-4.9.1.jar
+/Users/bertty/.m2/repository/org/antlr/antlr-runtime/3.5.2/antlr-runtime-3.5.2.jar
+/Users/bertty/.m2/repository/org/antlr/ST4/4.3/ST4-4.3.jar
+/Users/bertty/.m2/repository/org/abego/treelayout/org.abego.treelayout.core/1.0.3/org.abego.treelayout.core-1.0.3.jar
+/Users/bertty/.m2/repository/org/glassfish/javax.json/1.0.4/javax.json-1.0.4.jar
+/Users/bertty/.m2/repository/com/ibm/icu/icu4j/61.1/icu4j-61.1.jar com.intellij.rt.junit.JUnitStarter -ideVersion5 -junit5 org.apache.wayang.api.ApiTest,testReadMapCollect
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala
deleted file mode 100644
index c6e5dcb..0000000
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuantaBuilder.scala
+++ /dev/null
@@ -1,440 +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.wayang.api
-
-
-import java.util.function.{Consumer, IntUnaryOperator, Function => JavaFunction}
-import java.util.{Collection => JavaCollection}
-import de.hpi.isg.profiledb.store.model.Experiment
-import org.apache.wayang.api.dataquantabuilder.{BasicDataQuantaBuilder, CartesianDataQuantaBuilder, CoGroupDataQuantaBuilder, CountDataQuantaBuilder, CustomOperatorDataQuantaBuilder, DistinctDataQuantaBuilder, DoWhileDataQuantaBuilder, FilterDataQuantaBuilder, FlatMapDataQuantaBuilder, GlobalGroupDataQuantaBuilder, GlobalReduceDataQuantaBuilder, GroupByDataQuantaBuilder, IntersectDataQuantaBuilder, JoinDataQuantaBuilder, KeyedDataQuantaBuilder, MapDataQuantaBuilder, MapPartitionsDataQuan [...]
-import org.apache.wayang.api.graph.{Edge, EdgeDataQuantaBuilder, EdgeDataQuantaBuilderDecorator}
-import org.apache.wayang.api.util.{DataQuantaBuilderCache, TypeTrap}
-import org.apache.wayang.basic.data.{Record, Tuple2 => RT2}
-import org.apache.wayang.basic.operators.{GlobalReduceOperator, LocalCallbackSink, MapOperator, SampleOperator}
-import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBiFunction, SerializableBinaryOperator, SerializableFunction, SerializablePredicate}
-import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
-import org.apache.wayang.core.optimizer.cardinality.CardinalityEstimator
-import org.apache.wayang.core.optimizer.costs.{LoadEstimator, LoadProfile, LoadProfileEstimator}
-import org.apache.wayang.core.plan.wayangplan.{Operator, OutputSlot, UnarySource, WayangPlan}
-import org.apache.wayang.core.platform.Platform
-import org.apache.wayang.core.types.DataSetType
-import org.apache.wayang.core.util.{Logging, ReflectionUtils, WayangCollections, Tuple => WayangTuple}
-
-import scala.collection.mutable.ListBuffer
-import scala.reflect.ClassTag
-
-/**
-  * Trait/interface for builders of [[DataQuanta]]. The purpose of the builders is to provide a convenient
-  * Java API for Wayang that compensates for lacking default and named arguments.
-  */
-trait DataQuantaBuilder[+This <: DataQuantaBuilder[_, Out], Out] extends Logging {
-
-  /**
-    * The type of the [[DataQuanta]] to be built.
-    */
-  protected[api] def outputTypeTrap: TypeTrap
-
-  /**
-    * Provide a [[JavaPlanBuilder]] to which this instance is associated.
-    */
-  protected[api] implicit def javaPlanBuilder: JavaPlanBuilder
-
-  /**
-    * Set a name for the [[DataQuanta]] and its associated [[org.apache.wayang.core.plan.wayangplan.Operator]]s.
-    *
-    * @param name the name
-    * @return this instance
-    */
-  def withName(name: String): This
-
-  /**
-    * Set an [[Experiment]] for the currently built [[org.apache.wayang.core.api.Job]].
-    *
-    * @param experiment the [[Experiment]]
-    * @return this instance
-    */
-  def withExperiment(experiment: Experiment): This
-
-  /**
-    * Explicitly set an output [[DataSetType]] for the currently built [[DataQuanta]]. Note that it is not
-    * always necessary to set it and that it can be inferred in some situations.
-    *
-    * @param outputType the output [[DataSetType]]
-    * @return this instance
-    */
-  def withOutputType(outputType: DataSetType[Out]): This
-
-  /**
-    * Explicitly set an output [[Class]] for the currently built [[DataQuanta]]. Note that it is not
-    * always necessary to set it and that it can be inferred in some situations.
-    *
-    * @param cls the output [[Class]]
-    * @return this instance
-    */
-  def withOutputClass(cls: Class[Out]): This
-
-  /**
-    * Register a broadcast with the [[DataQuanta]] to be built
-    *
-    * @param sender        a [[DataQuantaBuilder]] constructing the broadcasted [[DataQuanta]]
-    * @param broadcastName the name of the broadcast
-    * @return this instance
-    */
-  def withBroadcast[Sender <: DataQuantaBuilder[_, _]](sender: Sender, broadcastName: String): This
-
-  /**
-    * Set a [[CardinalityEstimator]] for the currently built [[DataQuanta]].
-    *
-    * @param cardinalityEstimator the [[CardinalityEstimator]]
-    * @return this instance
-    */
-  def withCardinalityEstimator(cardinalityEstimator: CardinalityEstimator): This
-
-  /**
-    * Add a target [[Platform]] on which the currently built [[DataQuanta]] should be calculated. Can be invoked
-    * multiple times to set multiple possilbe target [[Platform]]s or not at all to impose no restrictions.
-    *
-    * @param platform the [[CardinalityEstimator]]
-    * @return this instance
-    */
-  def withTargetPlatform(platform: Platform): This
-
-  /**
-    * Register the JAR file containing the given [[Class]] with the currently built [[org.apache.wayang.core.api.Job]].
-    *
-    * @param cls the [[Class]]
-    * @return this instance
-    */
-  def withUdfJarOf(cls: Class[_]): This
-
-  /**
-    * Register a JAR file with the currently built [[org.apache.wayang.core.api.Job]].
-    *
-    * @param path the path of the JAR file
-    * @return this instance
-    */
-  def withUdfJar(path: String): This
-
-  /**
-    * Provide a [[ClassTag]] for the constructed [[DataQuanta]].
-    *
-    * @return the [[ClassTag]]
-    */
-  protected[api] implicit def classTag: ClassTag[Out] = ClassTag(outputTypeTrap.typeClass)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[MapOperator]].
-    *
-    * @param udf the UDF for the [[MapOperator]]
-    * @return a [[MapDataQuantaBuilder]]
-    */
-  def map[NewOut](udf: SerializableFunction[Out, NewOut]) = new MapDataQuantaBuilder(this, udf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[MapOperator]] with a [[org.apache.wayang.basic.function.ProjectionDescriptor]].
-    *
-    * @param fieldNames field names for the [[org.apache.wayang.basic.function.ProjectionDescriptor]]
-    * @return a [[MapDataQuantaBuilder]]
-    */
-  def project[NewOut](fieldNames: Array[String]) = new ProjectionDataQuantaBuilder(this, fieldNames)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.FilterOperator]].
-    *
-    * @param udf filter UDF
-    * @return a [[FilterDataQuantaBuilder]]
-    */
-  def filter(udf: SerializablePredicate[Out]) = new FilterDataQuantaBuilder(this, udf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.FlatMapOperator]].
-    *
-    * @param udf the UDF for the [[org.apache.wayang.basic.operators.FlatMapOperator]]
-    * @return a [[FlatMapDataQuantaBuilder]]
-    */
-  def flatMap[NewOut](udf: SerializableFunction[Out, java.lang.Iterable[NewOut]]) = new FlatMapDataQuantaBuilder(this, udf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.MapPartitionsOperator]].
-    *
-    * @param udf the UDF for the [[org.apache.wayang.basic.operators.MapPartitionsOperator]]
-    * @return a [[MapPartitionsDataQuantaBuilder]]
-    */
-  def mapPartitions[NewOut](udf: SerializableFunction[java.lang.Iterable[Out], java.lang.Iterable[NewOut]]) =
-    new MapPartitionsDataQuantaBuilder(this, udf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.SampleOperator]].
-    *
-    * @param sampleSize the absolute size of the sample
-    * @return a [[SampleDataQuantaBuilder]]
-    */
-  def sample(sampleSize: Int): SampleDataQuantaBuilder[Out] = this.sample(new IntUnaryOperator {
-    override def applyAsInt(operand: Int): Int = sampleSize
-  })
-
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.SampleOperator]].
-    *
-    * @param sampleSizeFunction the absolute size of the sample as a function of the current iteration number
-    * @return a [[SampleDataQuantaBuilder]]
-    */
-  def sample(sampleSizeFunction: IntUnaryOperator) = new SampleDataQuantaBuilder[Out](this, sampleSizeFunction)
-
-  /**
-    * Annotates a key to this instance.
-    * @param keyExtractor extracts the key from the data quanta
-    * @return a [[KeyedDataQuantaBuilder]]
-    */
-  def keyBy[Key](keyExtractor: SerializableFunction[Out, Key]) = new KeyedDataQuantaBuilder[Out, Key](this, keyExtractor)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[GlobalReduceOperator]].
-    *
-    * @param udf the UDF for the [[GlobalReduceOperator]]
-    * @return a [[GlobalReduceDataQuantaBuilder]]
-    */
-  def reduce(udf: SerializableBinaryOperator[Out]) = new GlobalReduceDataQuantaBuilder(this, udf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.ReduceByOperator]].
-    *
-    * @param keyUdf the key UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
-    * @param udf    the UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
-    * @return a [[ReduceByDataQuantaBuilder]]
-    */
-  def reduceByKey[Key](keyUdf: SerializableFunction[Out, Key], udf: SerializableBinaryOperator[Out]) =
-    new ReduceByDataQuantaBuilder(this, keyUdf, udf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]].
-    *
-    * @param keyUdf the key UDF for the [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]]
-    * @return a [[GroupByDataQuantaBuilder]]
-    */
-  def groupByKey[Key](keyUdf: SerializableFunction[Out, Key]) =
-    new GroupByDataQuantaBuilder(this, keyUdf)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.GlobalMaterializedGroupOperator]].
-    *
-    * @return a [[GlobalGroupDataQuantaBuilder]]
-    */
-  def group() = new GlobalGroupDataQuantaBuilder(this)
-
-  /**
-    * Feed the built [[DataQuanta]] of this and the given instance into a
-    * [[org.apache.wayang.basic.operators.UnionAllOperator]].
-    *
-    * @param that the other [[DataQuantaBuilder]] to union with
-    * @return a [[UnionDataQuantaBuilder]]
-    */
-  def union(that: DataQuantaBuilder[_, Out]) = new UnionDataQuantaBuilder(this, that)
-
-  /**
-    * Feed the built [[DataQuanta]] of this and the given instance into a
-    * [[org.apache.wayang.basic.operators.IntersectOperator]].
-    *
-    * @param that the other [[DataQuantaBuilder]] to intersect with
-    * @return an [[IntersectDataQuantaBuilder]]
-    */
-  def intersect(that: DataQuantaBuilder[_, Out]) = new IntersectDataQuantaBuilder(this, that)
-
-  /**
-    * Feed the built [[DataQuanta]] of this and the given instance into a
-    * [[org.apache.wayang.basic.operators.JoinOperator]].
-    *
-    * @param thisKeyUdf the key extraction UDF for this instance
-    * @param that       the other [[DataQuantaBuilder]] to join with
-    * @param thatKeyUdf the key extraction UDF for `that` instance
-    * @return a [[JoinDataQuantaBuilder]]
-    */
-  def join[ThatOut, Key](thisKeyUdf: SerializableFunction[Out, Key],
-                         that: DataQuantaBuilder[_, ThatOut],
-                         thatKeyUdf: SerializableFunction[ThatOut, Key]) =
-    new JoinDataQuantaBuilder(this, that, thisKeyUdf, thatKeyUdf)
-
-  /**
-    * Feed the built [[DataQuanta]] of this and the given instance into a
-    * [[org.apache.wayang.basic.operators.CoGroupOperator]].
-    *
-    * @param thisKeyUdf the key extraction UDF for this instance
-    * @param that       the other [[DataQuantaBuilder]] to join with
-    * @param thatKeyUdf the key extraction UDF for `that` instance
-    * @return a [[CoGroupDataQuantaBuilder]]
-    */
-  def coGroup[ThatOut, Key](thisKeyUdf: SerializableFunction[Out, Key],
-                         that: DataQuantaBuilder[_, ThatOut],
-                         thatKeyUdf: SerializableFunction[ThatOut, Key]) =
-    new CoGroupDataQuantaBuilder(this, that, thisKeyUdf, thatKeyUdf)
-
-
-  /**
-    * Feed the built [[DataQuanta]] of this and the given instance into a
-    * [[org.apache.wayang.basic.operators.SortOperator]].
-    *
-    * @param keyUdf the key extraction UDF for this instance
-    * @return a [[SortDataQuantaBuilder]]
-    */
-  def sort[Key](keyUdf: SerializableFunction[Out, Key]) =
-    new SortDataQuantaBuilder(this, keyUdf)
-
-  /**
-    * Feed the built [[DataQuanta]] of this and the given instance into a
-    * [[org.apache.wayang.basic.operators.CartesianOperator]].
-    *
-    * @return a [[CartesianDataQuantaBuilder]]
-    */
-  def cartesian[ThatOut](that: DataQuantaBuilder[_, ThatOut]) = new CartesianDataQuantaBuilder(this, that)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.ZipWithIdOperator]].
-    *
-    * @return a [[ZipWithIdDataQuantaBuilder]] representing the [[org.apache.wayang.basic.operators.ZipWithIdOperator]]'s output
-    */
-  def zipWithId = new ZipWithIdDataQuantaBuilder(this)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.DistinctOperator]].
-    *
-    * @return a [[DistinctDataQuantaBuilder]] representing the [[org.apache.wayang.basic.operators.DistinctOperator]]'s output
-    */
-  def distinct = new DistinctDataQuantaBuilder(this)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.CountOperator]].
-    *
-    * @return a [[CountDataQuantaBuilder]] representing the [[org.apache.wayang.basic.operators.CountOperator]]'s output
-    */
-  def count = new CountDataQuantaBuilder(this)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.DoWhileOperator]].
-    *
-    * @return a [[DoWhileDataQuantaBuilder]]
-    */
-  def doWhile[Conv](conditionUdf: SerializablePredicate[JavaCollection[Conv]],
-                    bodyBuilder: JavaFunction[DataQuantaBuilder[_, Out], WayangTuple[DataQuantaBuilder[_, Out], DataQuantaBuilder[_, Conv]]]) =
-    new DoWhileDataQuantaBuilder(this, conditionUdf.asInstanceOf[SerializablePredicate[JavaCollection[Conv]]], bodyBuilder)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.RepeatOperator]].
-    *
-    * @return a [[DoWhileDataQuantaBuilder]]
-    */
-  def repeat(numRepetitions: Int, bodyBuilder: JavaFunction[DataQuantaBuilder[_, Out], DataQuantaBuilder[_, Out]]) =
-    new RepeatDataQuantaBuilder(this, numRepetitions, bodyBuilder)
-
-  /**
-    * Feed the built [[DataQuanta]] into a custom [[Operator]] with a single [[org.apache.wayang.core.plan.wayangplan.InputSlot]]
-    * and a single [[OutputSlot]].
-    *
-    * @param operator the custom [[Operator]]
-    * @tparam T the type of the output [[DataQuanta]]
-    * @return a [[CustomOperatorDataQuantaBuilder]]
-    */
-  def customOperator[T](operator: Operator) = {
-    assert(operator.getNumInputs == 1, "customOperator(...) only allows for operators with a single input.")
-    assert(operator.getNumOutputs == 1, "customOperator(...) only allows for operators with a single output.")
-    new CustomOperatorDataQuantaBuilder[T](operator, 0, new DataQuantaBuilderCache, this)
-  }
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[LocalCallbackSink]] that collects all data quanta locally. This triggers
-    * execution of the constructed [[WayangPlan]].
-    *
-    * @return the collected data quanta
-    */
-  def collect(): JavaCollection[Out] = {
-    import scala.collection.JavaConversions._
-    this.dataQuanta().collect()
-  }
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[JavaFunction]] that runs locally. This triggers
-    * execution of the constructed [[WayangPlan]].
-    *
-    * @param f the [[JavaFunction]]
-    * @return the collected data quanta
-    */
-  def forEach(f: Consumer[Out]): Unit = this.dataQuanta().foreachJava(f)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.TextFileSink]]. This triggers
-    * execution of the constructed [[WayangPlan]].
-    *
-    * @param url     the URL of the file to be written
-    * @param jobName optional name for the [[WayangPlan]]
-    * @return the collected data quanta
-    */
-  def writeTextFile(url: String, formatterUdf: SerializableFunction[Out, String], jobName: String): Unit =
-    this.writeTextFile(url, formatterUdf, jobName, null)
-
-  /**
-    * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.TextFileSink]]. This triggers
-    * execution of the constructed [[WayangPlan]].
-    *
-    * @param url the URL of the file to be written
-    * @return the collected data quanta
-    */
-  def writeTextFile(url: String,
-                    formatterUdf: SerializableFunction[Out, String],
-                    jobName: String,
-                    udfLoadProfileEstimator: LoadProfileEstimator): Unit = {
-    this.javaPlanBuilder.withJobName(jobName)
-    this.dataQuanta().writeTextFileJava(url, formatterUdf, udfLoadProfileEstimator)
-  }
-
-  /**
-    * Enriches the set of operations to [[Record]]-based ones. This instances must deal with data quanta of
-    * type [[Record]], though. Because of Java's type erasure, we need to leave it up to you whether this
-    * operation is applicable.
-    *
-    * @return a [[RecordDataQuantaBuilder]]
-    */
-  def asRecords[T <: RecordDataQuantaBuilder[T]]: RecordDataQuantaBuilder[T] = {
-    this match {
-      case records: RecordDataQuantaBuilder[_] => records.asInstanceOf[RecordDataQuantaBuilder[T]]
-      case _ => new RecordDataQuantaBuilderDecorator(this.asInstanceOf[DataQuantaBuilder[_, Record]])
-    }
-  }
-
-  /**
-    * Enriches the set of operations to [[Edge]]-based ones. This instances must deal with data quanta of
-    * type [[Edge]], though. Because of Java's type erasure, we need to leave it up to you whether this
-    * operation is applicable.
-    *
-    * @return a [[EdgeDataQuantaBuilder]]
-    */
-  def asEdges[T <: EdgeDataQuantaBuilder[T]]: EdgeDataQuantaBuilder[T] = {
-    this match {
-      case edges: RecordDataQuantaBuilder[_] => edges.asInstanceOf[EdgeDataQuantaBuilder[T]]
-      case _ => new EdgeDataQuantaBuilderDecorator(this.asInstanceOf[DataQuantaBuilder[_, Edge]])
-    }
-  }
-
-  /**
-    * Get or create the [[DataQuanta]] built by this instance.
-    *
-    * @return the [[DataQuanta]]
-    */
-  protected[api] def dataQuanta(): DataQuanta[Out]
-
-}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala
index 2c88f58..bcb916f 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/JavaPlanBuilder.scala
@@ -21,7 +21,8 @@ package org.apache.wayang.api
 import java.util.{Collection => JavaCollection}
 import de.hpi.isg.profiledb.store.model.Experiment
 import org.apache.commons.lang3.Validate
-import org.apache.wayang.api.dataquantabuilder.{CustomOperatorDataQuantaBuilder, LoadCollectionDataQuantaBuilder, UnarySourceDataQuantaBuilder}
+import org.apache.wayang.api.dataquanta.builder.{CustomOperatorDataQuantaBuilder, LoadCollectionDataQuantaBuilder, UnarySourceDataQuantaBuilder}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.DataQuantaBuilderCache
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.operators.{TableSource, TextFileSource}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/PlanBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/PlanBuilder.scala
index 8600e23..1c59ca9 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/PlanBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/PlanBuilder.scala
@@ -21,6 +21,7 @@ package org.apache.wayang.api
 import de.hpi.isg.profiledb.store.model.Experiment
 import org.apache.commons.lang3.Validate
 import org.apache.wayang.api
+import org.apache.wayang.api.dataquanta.DataQuanta
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.operators.{CollectionSource, TableSource, TextFileSource}
 import org.apache.wayang.core.api.WayangContext
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuanta.scala
index cc95e39..61c6909 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuanta.scala
@@ -18,6 +18,7 @@
 
 package org.apache.wayang.api
 
+import org.apache.wayang.api.dataquanta.DataQuanta
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.function.ProjectionDescriptor
 import org.apache.wayang.basic.operators.MapOperator
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala
index 5dc045b..0f1c1e2 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/RecordDataQuantaBuilder.scala
@@ -18,7 +18,8 @@
 
 package org.apache.wayang.api
 
-import org.apache.wayang.api.dataquantabuilder.BasicDataQuantaBuilder
+import org.apache.wayang.api.dataquanta.builder.BasicDataQuantaBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.DataQuantaBuilderDecorator
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.function.ProjectionDescriptor
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
similarity index 90%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuanta.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
index 89722cc..a614467 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/DataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
@@ -1,29 +1,28 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.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,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
  */
 
-package org.apache.wayang.api
-
-import _root_.java.lang.{Iterable => JavaIterable}
-import _root_.java.util.function.{Consumer, IntUnaryOperator, BiFunction => JavaBiFunction, Function => JavaFunction}
-import _root_.java.util.{Collection => JavaCollection}
+package org.apache.wayang.api.dataquanta
 
 import de.hpi.isg.profiledb.store.model.Experiment
 import org.apache.commons.lang3.Validate
+import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, groupedDataSetType, groupedDataUnitType, toConsumer, toSerializableBinaryOperator, toSerializableFlatteningFunction, toSerializableFunction, toSerializablePartitionFunction, toSerializablePredicate}
+import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
 import org.apache.wayang.basic.function.ProjectionDescriptor
 import org.apache.wayang.basic.operators._
 import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction, SerializablePredicate}
@@ -34,8 +33,10 @@ import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
 import org.apache.wayang.core.plan.wayangplan._
 import org.apache.wayang.core.platform.Platform
 import org.apache.wayang.core.util.{Tuple => WayangTuple}
-import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
 
+import java.lang.{Iterable => JavaIterable}
+import java.util.function.{Consumer, IntUnaryOperator, BiFunction => JavaBiFunction, Function => JavaFunction}
+import java.util.{Collection => JavaCollection}
 import scala.collection.JavaConversions
 import scala.collection.JavaConversions._
 import scala.reflect._
@@ -866,65 +867,6 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
 
 }
 
-/**
-  * This class provides operations on [[DataQuanta]] with additional operations.
-  */
-class KeyedDataQuanta[Out: ClassTag, Key: ClassTag](val dataQuanta: DataQuanta[Out],
-                                                    val keyExtractor: SerializableFunction[Out, Key]) {
-
-  /**
-    * Performs a join. The join fields are governed by the [[KeyedDataQuanta]]'s keys.
-    *
-    * @param that the other [[KeyedDataQuanta]] to join with
-    * @return the join product [[DataQuanta]]
-    */
-  def join[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]):
-  DataQuanta[WayangTuple2[Out, ThatOut]] =
-    dataQuanta.joinJava[ThatOut, Key](this.keyExtractor, that.dataQuanta, that.keyExtractor)
-
-  /**
-    * Performs a co-group. The grouping fields are governed by the [[KeyedDataQuanta]]'s keys.
-    *
-    * @param that the other [[KeyedDataQuanta]] to co-group with
-    * @return the co-grouped [[DataQuanta]]
-    */
-  def coGroup[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]):
-  DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] =
-    dataQuanta.coGroupJava[ThatOut, Key](this.keyExtractor, that.dataQuanta, that.keyExtractor)
-
-}
-
-/**
-  * This class amends joined [[DataQuanta]] with additional operations.
-  */
-class JoinedDataQuanta[Out0: ClassTag, Out1: ClassTag]
-(val dataQuanta: DataQuanta[WayangTuple2[Out0, Out1]]) {
-
-  /**
-    * Assembles a new element from a join product tuple.
-    *
-    * @param udf     creates the output data quantum from two joinable data quanta
-    * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
-    * @return the join product [[DataQuanta]]
-    */
-  def assemble[NewOut: ClassTag](udf: (Out0, Out1) => NewOut,
-                                 udfLoad: LoadProfileEstimator = null):
-  DataQuanta[NewOut] =
-    dataQuanta.map(joinTuple => udf.apply(joinTuple.field0, joinTuple.field1), udfLoad)
-
-  /**
-    * Assembles a new element from a join product tuple.
-    *
-    * @param assembler creates the output data quantum from two joinable data quanta
-    * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
-    * @return the join product [[DataQuanta]]
-    */
-  def assembleJava[NewOut: ClassTag](assembler: JavaBiFunction[Out0, Out1, NewOut],
-                                     udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] =
-    dataQuanta.map(join => assembler.apply(join.field0, join.field1), udfLoad)
-
-}
-
 object DataQuanta {
 
   def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuanta[_] =
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaBuilder.scala
new file mode 100644
index 0000000..c0a8e8e
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaBuilder.scala
@@ -0,0 +1,441 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquanta
+
+import de.hpi.isg.profiledb.store.model.Experiment
+import org.apache.wayang.api.dataquanta.builder.{CartesianDataQuantaBuilder, CoGroupDataQuantaBuilder, CountDataQuantaBuilder, CustomOperatorDataQuantaBuilder, DistinctDataQuantaBuilder, DoWhileDataQuantaBuilder, FilterDataQuantaBuilder, FlatMapDataQuantaBuilder, GlobalGroupDataQuantaBuilder, GlobalReduceDataQuantaBuilder, GroupByDataQuantaBuilder, IntersectDataQuantaBuilder, JoinDataQuantaBuilder, KeyedDataQuantaBuilder, MapDataQuantaBuilder, MapPartitionsDataQuantaBuilder, ProjectionDa [...]
+import org.apache.wayang.api.graph.{Edge, EdgeDataQuantaBuilder, EdgeDataQuantaBuilderDecorator}
+import org.apache.wayang.api.util.{DataQuantaBuilderCache, TypeTrap}
+import org.apache.wayang.api.{JavaPlanBuilder, RecordDataQuantaBuilder, RecordDataQuantaBuilderDecorator}
+import org.apache.wayang.basic.data.Record
+import org.apache.wayang.basic.operators.{GlobalReduceOperator, LocalCallbackSink, MapOperator}
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction, SerializablePredicate}
+import org.apache.wayang.core.optimizer.cardinality.CardinalityEstimator
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
+import org.apache.wayang.core.plan.wayangplan.{Operator, OutputSlot, WayangPlan}
+import org.apache.wayang.core.platform.Platform
+import org.apache.wayang.core.types.DataSetType
+import org.apache.wayang.core.util.Logging
+import org.apache.wayang.core.util.{Tuple => WayangTuple}
+
+import java.util.function.{Consumer, IntUnaryOperator, Function => JavaFunction}
+import java.util.{Collection => JavaCollection}
+import scala.reflect.ClassTag
+
+/**
+ * Trait/interface for builders of [[DataQuanta]]. The purpose of the builders is to provide a convenient
+ * Java API for Wayang that compensates for lacking default and named arguments.
+ */
+trait DataQuantaBuilder[+This <: DataQuantaBuilder[_, Out], Out] extends Logging {
+
+  /**
+   * The type of the [[DataQuanta]] to be built.
+   */
+  protected[api] def outputTypeTrap: TypeTrap
+
+  /**
+   * Provide a [[JavaPlanBuilder]] to which this instance is associated.
+   */
+  protected[api] implicit def javaPlanBuilder: JavaPlanBuilder
+
+  /**
+   * Set a name for the [[DataQuanta]] and its associated [[org.apache.wayang.core.plan.wayangplan.Operator]]s.
+   *
+   * @param name the name
+   * @return this instance
+   */
+  def withName(name: String): This
+
+  /**
+   * Set an [[Experiment]] for the currently built [[org.apache.wayang.core.api.Job]].
+   *
+   * @param experiment the [[Experiment]]
+   * @return this instance
+   */
+  def withExperiment(experiment: Experiment): This
+
+  /**
+   * Explicitly set an output [[DataSetType]] for the currently built [[DataQuanta]]. Note that it is not
+   * always necessary to set it and that it can be inferred in some situations.
+   *
+   * @param outputType the output [[DataSetType]]
+   * @return this instance
+   */
+  def withOutputType(outputType: DataSetType[Out]): This
+
+  /**
+   * Explicitly set an output [[Class]] for the currently built [[DataQuanta]]. Note that it is not
+   * always necessary to set it and that it can be inferred in some situations.
+   *
+   * @param cls the output [[Class]]
+   * @return this instance
+   */
+  def withOutputClass(cls: Class[Out]): This
+
+  /**
+   * Register a broadcast with the [[DataQuanta]] to be built
+   *
+   * @param sender        a [[DataQuantaBuilder]] constructing the broadcasted [[DataQuanta]]
+   * @param broadcastName the name of the broadcast
+   * @return this instance
+   */
+  def withBroadcast[Sender <: DataQuantaBuilder[_, _]](sender: Sender, broadcastName: String): This
+
+  /**
+   * Set a [[CardinalityEstimator]] for the currently built [[DataQuanta]].
+   *
+   * @param cardinalityEstimator the [[CardinalityEstimator]]
+   * @return this instance
+   */
+  def withCardinalityEstimator(cardinalityEstimator: CardinalityEstimator): This
+
+  /**
+   * Add a target [[Platform]] on which the currently built [[DataQuanta]] should be calculated. Can be invoked
+   * multiple times to set multiple possilbe target [[Platform]]s or not at all to impose no restrictions.
+   *
+   * @param platform the [[CardinalityEstimator]]
+   * @return this instance
+   */
+  def withTargetPlatform(platform: Platform): This
+
+  /**
+   * Register the JAR file containing the given [[Class]] with the currently built [[org.apache.wayang.core.api.Job]].
+   *
+   * @param cls the [[Class]]
+   * @return this instance
+   */
+  def withUdfJarOf(cls: Class[_]): This
+
+  /**
+   * Register a JAR file with the currently built [[org.apache.wayang.core.api.Job]].
+   *
+   * @param path the path of the JAR file
+   * @return this instance
+   */
+  def withUdfJar(path: String): This
+
+  /**
+   * Provide a [[ClassTag]] for the constructed [[DataQuanta]].
+   *
+   * @return the [[ClassTag]]
+   */
+  protected[api] implicit def classTag: ClassTag[Out] = ClassTag(outputTypeTrap.typeClass)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[MapOperator]].
+   *
+   * @param udf the UDF for the [[MapOperator]]
+   * @return a [[MapDataQuantaBuilder]]
+   */
+  def map[NewOut](udf: SerializableFunction[Out, NewOut]) = new MapDataQuantaBuilder(this, udf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[MapOperator]] with a [[org.apache.wayang.basic.function.ProjectionDescriptor]].
+   *
+   * @param fieldNames field names for the [[org.apache.wayang.basic.function.ProjectionDescriptor]]
+   * @return a [[MapDataQuantaBuilder]]
+   */
+  def project[NewOut](fieldNames: Array[String]) = new ProjectionDataQuantaBuilder(this, fieldNames)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.FilterOperator]].
+   *
+   * @param udf filter UDF
+   * @return a [[FilterDataQuantaBuilder]]
+   */
+  def filter(udf: SerializablePredicate[Out]) = new FilterDataQuantaBuilder(this, udf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.FlatMapOperator]].
+   *
+   * @param udf the UDF for the [[org.apache.wayang.basic.operators.FlatMapOperator]]
+   * @return a [[FlatMapDataQuantaBuilder]]
+   */
+  def flatMap[NewOut](udf: SerializableFunction[Out, java.lang.Iterable[NewOut]]) = new FlatMapDataQuantaBuilder(this, udf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.MapPartitionsOperator]].
+   *
+   * @param udf the UDF for the [[org.apache.wayang.basic.operators.MapPartitionsOperator]]
+   * @return a [[MapPartitionsDataQuantaBuilder]]
+   */
+  def mapPartitions[NewOut](udf: SerializableFunction[java.lang.Iterable[Out], java.lang.Iterable[NewOut]]) =
+    new MapPartitionsDataQuantaBuilder(this, udf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.SampleOperator]].
+   *
+   * @param sampleSize the absolute size of the sample
+   * @return a [[SampleDataQuantaBuilder]]
+   */
+  def sample(sampleSize: Int): SampleDataQuantaBuilder[Out] = this.sample(new IntUnaryOperator {
+    override def applyAsInt(operand: Int): Int = sampleSize
+  })
+
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.SampleOperator]].
+   *
+   * @param sampleSizeFunction the absolute size of the sample as a function of the current iteration number
+   * @return a [[SampleDataQuantaBuilder]]
+   */
+  def sample(sampleSizeFunction: IntUnaryOperator) = new SampleDataQuantaBuilder[Out](this, sampleSizeFunction)
+
+  /**
+   * Annotates a key to this instance.
+   *
+   * @param keyExtractor extracts the key from the data quanta
+   * @return a [[KeyedDataQuantaBuilder]]
+   */
+  def keyBy[Key](keyExtractor: SerializableFunction[Out, Key]) = new KeyedDataQuantaBuilder[Out, Key](this, keyExtractor)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[GlobalReduceOperator]].
+   *
+   * @param udf the UDF for the [[GlobalReduceOperator]]
+   * @return a [[GlobalReduceDataQuantaBuilder]]
+   */
+  def reduce(udf: SerializableBinaryOperator[Out]) = new GlobalReduceDataQuantaBuilder(this, udf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.ReduceByOperator]].
+   *
+   * @param keyUdf the key UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
+   * @param udf    the UDF for the [[org.apache.wayang.basic.operators.ReduceByOperator]]
+   * @return a [[ReduceByDataQuantaBuilder]]
+   */
+  def reduceByKey[Key](keyUdf: SerializableFunction[Out, Key], udf: SerializableBinaryOperator[Out]) =
+    new ReduceByDataQuantaBuilder(this, keyUdf, udf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]].
+   *
+   * @param keyUdf the key UDF for the [[org.apache.wayang.basic.operators.MaterializedGroupByOperator]]
+   * @return a [[GroupByDataQuantaBuilder]]
+   */
+  def groupByKey[Key](keyUdf: SerializableFunction[Out, Key]) =
+    new GroupByDataQuantaBuilder(this, keyUdf)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.GlobalMaterializedGroupOperator]].
+   *
+   * @return a [[GlobalGroupDataQuantaBuilder]]
+   */
+  def group() = new GlobalGroupDataQuantaBuilder(this)
+
+  /**
+   * Feed the built [[DataQuanta]] of this and the given instance into a
+   * [[org.apache.wayang.basic.operators.UnionAllOperator]].
+   *
+   * @param that the other [[DataQuantaBuilder]] to union with
+   * @return a [[UnionDataQuantaBuilder]]
+   */
+  def union(that: DataQuantaBuilder[_, Out]) = new UnionDataQuantaBuilder(this, that)
+
+  /**
+   * Feed the built [[DataQuanta]] of this and the given instance into a
+   * [[org.apache.wayang.basic.operators.IntersectOperator]].
+   *
+   * @param that the other [[DataQuantaBuilder]] to intersect with
+   * @return an [[IntersectDataQuantaBuilder]]
+   */
+  def intersect(that: DataQuantaBuilder[_, Out]) = new IntersectDataQuantaBuilder(this, that)
+
+  /**
+   * Feed the built [[DataQuanta]] of this and the given instance into a
+   * [[org.apache.wayang.basic.operators.JoinOperator]].
+   *
+   * @param thisKeyUdf the key extraction UDF for this instance
+   * @param that       the other [[DataQuantaBuilder]] to join with
+   * @param thatKeyUdf the key extraction UDF for `that` instance
+   * @return a [[JoinDataQuantaBuilder]]
+   */
+  def join[ThatOut, Key](thisKeyUdf: SerializableFunction[Out, Key],
+                         that: DataQuantaBuilder[_, ThatOut],
+                         thatKeyUdf: SerializableFunction[ThatOut, Key]) =
+    new JoinDataQuantaBuilder(this, that, thisKeyUdf, thatKeyUdf)
+
+  /**
+   * Feed the built [[DataQuanta]] of this and the given instance into a
+   * [[org.apache.wayang.basic.operators.CoGroupOperator]].
+   *
+   * @param thisKeyUdf the key extraction UDF for this instance
+   * @param that       the other [[DataQuantaBuilder]] to join with
+   * @param thatKeyUdf the key extraction UDF for `that` instance
+   * @return a [[CoGroupDataQuantaBuilder]]
+   */
+  def coGroup[ThatOut, Key](thisKeyUdf: SerializableFunction[Out, Key],
+                            that: DataQuantaBuilder[_, ThatOut],
+                            thatKeyUdf: SerializableFunction[ThatOut, Key]) =
+    new CoGroupDataQuantaBuilder(this, that, thisKeyUdf, thatKeyUdf)
+
+
+  /**
+   * Feed the built [[DataQuanta]] of this and the given instance into a
+   * [[org.apache.wayang.basic.operators.SortOperator]].
+   *
+   * @param keyUdf the key extraction UDF for this instance
+   * @return a [[SortDataQuantaBuilder]]
+   */
+  def sort[Key](keyUdf: SerializableFunction[Out, Key]) =
+    new SortDataQuantaBuilder(this, keyUdf)
+
+  /**
+   * Feed the built [[DataQuanta]] of this and the given instance into a
+   * [[org.apache.wayang.basic.operators.CartesianOperator]].
+   *
+   * @return a [[CartesianDataQuantaBuilder]]
+   */
+  def cartesian[ThatOut](that: DataQuantaBuilder[_, ThatOut]) = new CartesianDataQuantaBuilder(this, that)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.ZipWithIdOperator]].
+   *
+   * @return a [[ZipWithIdDataQuantaBuilder]] representing the [[org.apache.wayang.basic.operators.ZipWithIdOperator]]'s output
+   */
+  def zipWithId = new ZipWithIdDataQuantaBuilder(this)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.DistinctOperator]].
+   *
+   * @return a [[DistinctDataQuantaBuilder]] representing the [[org.apache.wayang.basic.operators.DistinctOperator]]'s output
+   */
+  def distinct = new DistinctDataQuantaBuilder(this)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.CountOperator]].
+   *
+   * @return a [[CountDataQuantaBuilder]] representing the [[org.apache.wayang.basic.operators.CountOperator]]'s output
+   */
+  def count = new CountDataQuantaBuilder(this)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.DoWhileOperator]].
+   *
+   * @return a [[DoWhileDataQuantaBuilder]]
+   */
+  def doWhile[Conv](conditionUdf: SerializablePredicate[JavaCollection[Conv]],
+                    bodyBuilder: JavaFunction[DataQuantaBuilder[_, Out], WayangTuple[DataQuantaBuilder[_, Out], DataQuantaBuilder[_, Conv]]]) =
+    new DoWhileDataQuantaBuilder(this, conditionUdf.asInstanceOf[SerializablePredicate[JavaCollection[Conv]]], bodyBuilder)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.RepeatOperator]].
+   *
+   * @return a [[DoWhileDataQuantaBuilder]]
+   */
+  def repeat(numRepetitions: Int, bodyBuilder: JavaFunction[DataQuantaBuilder[_, Out], DataQuantaBuilder[_, Out]]) =
+    new RepeatDataQuantaBuilder(this, numRepetitions, bodyBuilder)
+
+  /**
+   * Feed the built [[DataQuanta]] into a custom [[Operator]] with a single [[org.apache.wayang.core.plan.wayangplan.InputSlot]]
+   * and a single [[OutputSlot]].
+   *
+   * @param operator the custom [[Operator]]
+   * @tparam T the type of the output [[DataQuanta]]
+   * @return a [[CustomOperatorDataQuantaBuilder]]
+   */
+  def customOperator[T](operator: Operator) = {
+    assert(operator.getNumInputs == 1, "customOperator(...) only allows for operators with a single input.")
+    assert(operator.getNumOutputs == 1, "customOperator(...) only allows for operators with a single output.")
+    new CustomOperatorDataQuantaBuilder[T](operator, 0, new DataQuantaBuilderCache, this)
+  }
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[LocalCallbackSink]] that collects all data quanta locally. This triggers
+   * execution of the constructed [[WayangPlan]].
+   *
+   * @return the collected data quanta
+   */
+  def collect(): JavaCollection[Out] = {
+    import scala.collection.JavaConversions._
+    this.dataQuanta().collect()
+  }
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[JavaFunction]] that runs locally. This triggers
+   * execution of the constructed [[WayangPlan]].
+   *
+   * @param f the [[JavaFunction]]
+   * @return the collected data quanta
+   */
+  def forEach(f: Consumer[Out]): Unit = this.dataQuanta().foreachJava(f)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.TextFileSink]]. This triggers
+   * execution of the constructed [[WayangPlan]].
+   *
+   * @param url     the URL of the file to be written
+   * @param jobName optional name for the [[WayangPlan]]
+   * @return the collected data quanta
+   */
+  def writeTextFile(url: String, formatterUdf: SerializableFunction[Out, String], jobName: String): Unit =
+    this.writeTextFile(url, formatterUdf, jobName, null)
+
+  /**
+   * Feed the built [[DataQuanta]] into a [[org.apache.wayang.basic.operators.TextFileSink]]. This triggers
+   * execution of the constructed [[WayangPlan]].
+   *
+   * @param url the URL of the file to be written
+   * @return the collected data quanta
+   */
+  def writeTextFile(url: String,
+                    formatterUdf: SerializableFunction[Out, String],
+                    jobName: String,
+                    udfLoadProfileEstimator: LoadProfileEstimator): Unit = {
+    this.javaPlanBuilder.withJobName(jobName)
+    this.dataQuanta().writeTextFileJava(url, formatterUdf, udfLoadProfileEstimator)
+  }
+
+  /**
+   * Enriches the set of operations to [[Record]]-based ones. This instances must deal with data quanta of
+   * type [[Record]], though. Because of Java's type erasure, we need to leave it up to you whether this
+   * operation is applicable.
+   *
+   * @return a [[RecordDataQuantaBuilder]]
+   */
+  def asRecords[T <: RecordDataQuantaBuilder[T]]: RecordDataQuantaBuilder[T] = {
+    this match {
+      case records: RecordDataQuantaBuilder[_] => records.asInstanceOf[RecordDataQuantaBuilder[T]]
+      case _ => new RecordDataQuantaBuilderDecorator(this.asInstanceOf[DataQuantaBuilder[_, Record]])
+    }
+  }
+
+  /**
+   * Enriches the set of operations to [[Edge]]-based ones. This instances must deal with data quanta of
+   * type [[Edge]], though. Because of Java's type erasure, we need to leave it up to you whether this
+   * operation is applicable.
+   *
+   * @return a [[EdgeDataQuantaBuilder]]
+   */
+  def asEdges[T <: EdgeDataQuantaBuilder[T]]: EdgeDataQuantaBuilder[T] = {
+    this match {
+      case edges: RecordDataQuantaBuilder[_] => edges.asInstanceOf[EdgeDataQuantaBuilder[T]]
+      case _ => new EdgeDataQuantaBuilderDecorator(this.asInstanceOf[DataQuantaBuilder[_, Edge]])
+    }
+  }
+
+  /**
+   * Get or create the [[DataQuanta]] built by this instance.
+   *
+   * @return the [[DataQuanta]]
+   */
+  protected[api] def dataQuanta(): DataQuanta[Out]
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.scala
new file mode 100644
index 0000000..99768cc
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.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.wayang.api.dataquanta
+
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
+import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
+
+import scala.reflect.ClassTag
+
+/**
+ * This class amends joined [[DataQuanta]] with additional operations.
+ */
+class JoinedDataQuanta[Out0: ClassTag, Out1: ClassTag]
+(val dataQuanta: DataQuanta[WayangTuple2[Out0, Out1]]) {
+
+  /**
+   * Assembles a new element from a join product tuple.
+   *
+   * @param udf     creates the output data quantum from two joinable data quanta
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return the join product [[DataQuanta]]
+   */
+  def assemble[NewOut: ClassTag](udf: (Out0, Out1) => NewOut,
+                                 udfLoad: LoadProfileEstimator = null):
+  DataQuanta[NewOut] =
+    dataQuanta.map(joinTuple => udf.apply(joinTuple.field0, joinTuple.field1), udfLoad)
+
+  /**
+   * Assembles a new element from a join product tuple.
+   *
+   * @param assembler creates the output data quantum from two joinable data quanta
+   * @param udfLoad   optional [[LoadProfileEstimator]] for the `udf`
+   * @return the join product [[DataQuanta]]
+   */
+  def assembleJava[NewOut: ClassTag](assembler: JavaBiFunction[Out0, Out1, NewOut],
+                                     udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] =
+    dataQuanta.map(join => assembler.apply(join.field0, join.field1), udfLoad)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.scala
new file mode 100644
index 0000000..fb6e3f1
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.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.wayang.api.dataquanta
+
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
+
+import scala.reflect.ClassTag
+
+/**
+ * This class provides operations on [[DataQuanta]] with additional operations.
+ */
+class KeyedDataQuanta[Out: ClassTag, Key: ClassTag](val dataQuanta: DataQuanta[Out],
+                                                    val keyExtractor: SerializableFunction[Out, Key]) {
+
+  /**
+   * Performs a join. The join fields are governed by the [[KeyedDataQuanta]]'s keys.
+   *
+   * @param that the other [[KeyedDataQuanta]] to join with
+   * @return the join product [[DataQuanta]]
+   */
+  def join[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]):
+  DataQuanta[WayangTuple2[Out, ThatOut]] =
+    dataQuanta.joinJava[ThatOut, Key](this.keyExtractor, that.dataQuanta, that.keyExtractor)
+
+  /**
+   * Performs a co-group. The grouping fields are governed by the [[KeyedDataQuanta]]'s keys.
+   *
+   * @param that the other [[KeyedDataQuanta]] to co-group with
+   * @return the co-grouped [[DataQuanta]]
+   */
+  def coGroup[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]):
+  DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] =
+    dataQuanta.coGroupJava[ThatOut, Key](this.keyExtractor, that.dataQuanta, that.keyExtractor)
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/BasicDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/BasicDataQuantaBuilder.scala
similarity index 96%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/BasicDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/BasicDataQuantaBuilder.scala
index 9aad1a1..78cb3bb 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/BasicDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/BasicDataQuantaBuilder.scala
@@ -17,11 +17,12 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
 import de.hpi.isg.profiledb.store.model.Experiment
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.core.optimizer.cardinality.CardinalityEstimator
 import org.apache.wayang.core.platform.Platform
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CartesianDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CartesianDataQuantaBuilder.scala
similarity index 91%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CartesianDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CartesianDataQuantaBuilder.scala
index 40d4fae..fad5ff8 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CartesianDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CartesianDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
+import org.apache.wayang.api.{JavaPlanBuilder, dataSetType}
 import org.apache.wayang.basic.data.{Tuple2 => WT2}
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CartesianOperator]]s.
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CoGroupDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CoGroupDataQuantaBuilder.scala
similarity index 96%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CoGroupDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CoGroupDataQuantaBuilder.scala
index aa1d88a..79f2b5f 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CoGroupDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CoGroupDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
+import org.apache.wayang.api.{JavaPlanBuilder, dataSetType}
 import org.apache.wayang.basic.data.{Tuple2 => WT2}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 import org.apache.wayang.core.optimizer.costs.LoadEstimator
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CountDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CountDataQuantaBuilder.scala
similarity index 88%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CountDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CountDataQuantaBuilder.scala
index d8f7d18..9d51e7c 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CountDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CountDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
+import org.apache.wayang.api.{JavaPlanBuilder, dataSetType}
 
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.CountOperator]]s.
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CustomOperatorDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CustomOperatorDataQuantaBuilder.scala
similarity index 93%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CustomOperatorDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CustomOperatorDataQuantaBuilder.scala
index 5fca840..09c88f2 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/CustomOperatorDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/CustomOperatorDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.DataQuantaBuilderCache
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.core.plan.wayangplan.{Operator, OutputSlot, WayangPlan}
 
 /**
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DistinctDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/DistinctDataQuantaBuilder.scala
similarity index 90%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DistinctDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/DistinctDataQuantaBuilder.scala
index 9e8e044..f112388 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DistinctDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/DistinctDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.DistinctOperator]]s.
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DoWhileDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/DoWhileDataQuantaBuilder.scala
similarity index 96%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DoWhileDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/DoWhileDataQuantaBuilder.scala
index 34de7be..aabbb3a 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/DoWhileDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/DoWhileDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializablePredicate
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FakeDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FakeDataQuantaBuilder.scala
similarity index 90%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FakeDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FakeDataQuantaBuilder.scala
index 6b52c40..a8982d8 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FakeDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FakeDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 
 import scala.reflect.ClassTag
 
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FilterDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FilterDataQuantaBuilder.scala
similarity index 95%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FilterDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FilterDataQuantaBuilder.scala
index 3d33c80..ce546e3 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FilterDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FilterDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.basic.operators.MapOperator
 import org.apache.wayang.core.function.FunctionDescriptor.{SerializableFunction, SerializablePredicate}
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FlatMapDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FlatMapDataQuantaBuilder.scala
similarity index 95%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FlatMapDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FlatMapDataQuantaBuilder.scala
index 83b4383..19aadc2 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/FlatMapDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/FlatMapDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalGroupDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GlobalGroupDataQuantaBuilder.scala
similarity index 88%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalGroupDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GlobalGroupDataQuantaBuilder.scala
index abc1d9a..794d8c4 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalGroupDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GlobalGroupDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.GlobalMaterializedGroupOperator]]s.
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalReduceDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GlobalReduceDataQuantaBuilder.scala
similarity index 94%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalReduceDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GlobalReduceDataQuantaBuilder.scala
index c3efc95..dac4e4e 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GlobalReduceDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GlobalReduceDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableBinaryOperator
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GroupByDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GroupByDataQuantaBuilder.scala
similarity index 94%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GroupByDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GroupByDataQuantaBuilder.scala
index b61c71e..3531271 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/GroupByDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/GroupByDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/IntersectDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/IntersectDataQuantaBuilder.scala
similarity index 90%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/IntersectDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/IntersectDataQuantaBuilder.scala
index 6db8ffe..71fcf76 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/IntersectDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/IntersectDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.IntersectOperator]]s.
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/JoinDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/JoinDataQuantaBuilder.scala
similarity index 97%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/JoinDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/JoinDataQuantaBuilder.scala
index ab49ffb..4cfb938 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/JoinDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/JoinDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
+import org.apache.wayang.api.{JavaPlanBuilder, dataSetType}
 import org.apache.wayang.basic.data.{Tuple2 => WT2}
 import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBiFunction, SerializableFunction}
 import org.apache.wayang.core.optimizer.costs.LoadEstimator
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/KeyedDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/KeyedDataQuantaBuilder.scala
similarity index 92%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/KeyedDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/KeyedDataQuantaBuilder.scala
index 9ddc101..7e34472 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/KeyedDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/KeyedDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.DataQuantaBuilder
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 
 /**
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/LoadCollectionDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/LoadCollectionDataQuantaBuilder.scala
similarity index 91%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/LoadCollectionDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/LoadCollectionDataQuantaBuilder.scala
index 748416e..d5ee73f 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/LoadCollectionDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/LoadCollectionDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.core.types.DataSetType
 import org.apache.wayang.core.util.WayangCollections
 
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/MapDataQuantaBuilder.scala
similarity index 94%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/MapDataQuantaBuilder.scala
index deb15a9..f0927f9 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/MapDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.basic.operators.MapOperator
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapPartitionsDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/MapPartitionsDataQuantaBuilder.scala
similarity index 95%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapPartitionsDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/MapPartitionsDataQuantaBuilder.scala
index e686565..708cfc6 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/MapPartitionsDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/MapPartitionsDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ProjectionDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ProjectionDataQuantaBuilder.scala
similarity index 90%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ProjectionDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ProjectionDataQuantaBuilder.scala
index 28ede1e..9e3f7ce 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ProjectionDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ProjectionDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.MapOperator]]s with
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ReduceByDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ReduceByDataQuantaBuilder.scala
similarity index 96%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ReduceByDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ReduceByDataQuantaBuilder.scala
index ca14576..9dbd9bf 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ReduceByDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ReduceByDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction}
 import org.apache.wayang.core.optimizer.costs.{LoadProfile, LoadProfileEstimator}
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/RepeatDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/RepeatDataQuantaBuilder.scala
similarity index 92%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/RepeatDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/RepeatDataQuantaBuilder.scala
index f7170f7..5246823 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/RepeatDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/RepeatDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 
 import java.util.function.{Function => JavaFunction}
 
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SampleDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/SampleDataQuantaBuilder.scala
similarity index 94%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SampleDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/SampleDataQuantaBuilder.scala
index 0642fe6..19d0fea 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SampleDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/SampleDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.basic.operators.SampleOperator
 
 import java.util.function.IntUnaryOperator
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SortDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/SortDataQuantaBuilder.scala
similarity index 95%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SortDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/SortDataQuantaBuilder.scala
index d1af25a..ab6511d 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/SortDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/SortDataQuantaBuilder.scala
@@ -17,10 +17,11 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.TypeTrap
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
 import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
 import org.apache.wayang.core.optimizer.costs.LoadEstimator
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnarySourceDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/UnarySourceDataQuantaBuilder.scala
similarity index 89%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnarySourceDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/UnarySourceDataQuantaBuilder.scala
index d97b08b..9180e94 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnarySourceDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/UnarySourceDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.core.plan.wayangplan.UnarySource
 
 /**
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnionDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/UnionDataQuantaBuilder.scala
similarity index 90%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnionDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/UnionDataQuantaBuilder.scala
index 04fc6ac..aae2f5e 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/UnionDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/UnionDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 
 /**
  * [[DataQuantaBuilder]] implementation for [[org.apache.wayang.basic.operators.UnionAllOperator]]s.
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ZipWithIdDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ZipWithIdDataQuantaBuilder.scala
similarity index 89%
rename from wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ZipWithIdDataQuantaBuilder.scala
rename to wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ZipWithIdDataQuantaBuilder.scala
index ded640c..e5d246a 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquantabuilder/ZipWithIdDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/builder/ZipWithIdDataQuantaBuilder.scala
@@ -17,9 +17,10 @@
  *   under the License.
  */
 
-package org.apache.wayang.api.dataquantabuilder
+package org.apache.wayang.api.dataquanta.builder
 
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, dataSetType}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
+import org.apache.wayang.api.{JavaPlanBuilder, dataSetType}
 import org.apache.wayang.basic.data.{Tuple2 => WT2}
 
 /**
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuanta.scala
index 6ac73ac..99a1df6 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuanta.scala
@@ -19,6 +19,7 @@
 package org.apache.wayang.api.graph
 
 import org.apache.wayang.api._
+import org.apache.wayang.api.dataquanta.DataQuanta
 import org.apache.wayang.basic.data.Record
 import org.apache.wayang.basic.operators.{MapOperator, PageRankOperator}
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala
index 1eaec53..526fb6a 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/EdgeDataQuantaBuilder.scala
@@ -18,9 +18,10 @@
 
 package org.apache.wayang.api.graph
 
-import org.apache.wayang.api.dataquantabuilder.BasicDataQuantaBuilder
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 import org.apache.wayang.api.util.DataQuantaBuilderDecorator
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder, _}
+import org.apache.wayang.api.{JavaPlanBuilder,_ }
+import org.apache.wayang.api.dataquanta.builder.BasicDataQuantaBuilder
 import org.apache.wayang.basic.operators.PageRankOperator
 import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
 
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/package.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/package.scala
index e43fe81..a761e6a 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/package.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/graph/package.scala
@@ -18,6 +18,7 @@
 
 package org.apache.wayang.api
 
+import org.apache.wayang.api.dataquanta.DataQuanta
 import org.apache.wayang.basic.data.{Tuple2 => T2}
 
 /**
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala
index f17f5a5..7dfefd0 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala
@@ -18,9 +18,10 @@
 
 package org.apache.wayang
 
+import org.apache.wayang.api.dataquanta.{DataQuanta, JoinedDataQuanta}
+
 import _root_.java.lang.{Class => JavaClass, Iterable => JavaIterable}
 import _root_.java.util.function.{Consumer, ToLongBiFunction, ToLongFunction}
-
 import org.apache.wayang.basic.data.{Record, Tuple2 => WayangTuple2}
 import org.apache.wayang.core.api.WayangContext
 import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction, SerializablePredicate}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderCache.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderCache.scala
index 21decc2..456efb4 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderCache.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderCache.scala
@@ -18,10 +18,10 @@
 
 package org.apache.wayang.api.util
 
-import org.apache.wayang.api.DataQuanta
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
 
 /**
-  * Caches products of [[org.apache.wayang.api.DataQuantaBuilder]]s that need to be executed at once, e.g., because they
+  * Caches products of [[DataQuantaBuilder]]s that need to be executed at once, e.g., because they
   * belong to different [[org.apache.wayang.core.plan.wayangplan.OutputSlot]]s of the same custom [[org.apache.wayang.core.plan.wayangplan.Operator]].
   */
 class DataQuantaBuilderCache {
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderDecorator.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderDecorator.scala
index 8ea3dd2..a3b6b58 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderDecorator.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/util/DataQuantaBuilderDecorator.scala
@@ -19,7 +19,8 @@
 package org.apache.wayang.api.util
 
 import de.hpi.isg.profiledb.store.model.Experiment
-import org.apache.wayang.api.{DataQuanta, DataQuantaBuilder, JavaPlanBuilder}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaBuilder}
+import org.apache.wayang.api.JavaPlanBuilder
 import org.apache.wayang.core.optimizer.cardinality.CardinalityEstimator
 import org.apache.wayang.core.platform.Platform
 import org.apache.wayang.core.types.DataSetType
diff --git a/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java b/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java
index 71996a0..b21ea79 100644
--- a/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java
+++ b/wayang-api/wayang-api-scala-java/src/test/java/org/apache/wayang/api/JavaApiTest.java
@@ -18,8 +18,9 @@
 
 package org.apache.wayang.api;
 
-import org.apache.wayang.api.dataquantabuilder.GlobalReduceDataQuantaBuilder;
-import org.apache.wayang.api.dataquantabuilder.LoadCollectionDataQuantaBuilder;
+import org.apache.wayang.api.dataquanta.DataQuantaBuilder;
+import org.apache.wayang.api.dataquanta.builder.GlobalReduceDataQuantaBuilder;
+import org.apache.wayang.api.dataquanta.builder.LoadCollectionDataQuantaBuilder;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
diff --git a/wayang-api/wayang-api-scala-java/src/test/scala/org/apache/wayang/api/ApiTest.scala b/wayang-api/wayang-api-scala-java/src/test/scala/org/apache/wayang/api/ApiTest.scala
index cc05bd2..0e703c6 100644
--- a/wayang-api/wayang-api-scala-java/src/test/scala/org/apache/wayang/api/ApiTest.scala
+++ b/wayang-api/wayang-api-scala-java/src/test/scala/org/apache/wayang/api/ApiTest.scala
@@ -18,12 +18,13 @@
 
 package org.apache.wayang.api
 
+import org.apache.wayang.api.dataquanta.DataQuanta
+
 import java.io.File
 import java.net.URI
 import java.nio.file.{Files, Paths}
 import java.sql.{Connection, Statement}
 import java.util.function.Consumer
-
 import org.junit.{Assert, Test}
 import org.apache.wayang.basic.WayangBasics
 import org.apache.wayang.core.api.{Configuration, WayangContext}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
index 0839a6b..548f9e6 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/pom.xml
@@ -30,12 +30,45 @@
 
     <artifactId>wayang-hackit-api</artifactId>
 
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.wayang</groupId>
+                <artifactId>wayang-commons</artifactId>
+                <version>0.6.0-SNAPSHOT</version>
+                <type>pom</type>
+                <scope>import</scope>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.wayang</groupId>
             <artifactId>wayang-api-scala-java_2.11</artifactId>
             <version>0.6.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.wayang</groupId>
+            <artifactId>wayang-java</artifactId>
+            <version>0.6.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
 </project>
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/java/org/apache/wayang/api/JavaApiTest.java b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/java/org/apache/wayang/api/JavaApiTest.java
new file mode 100644
index 0000000..aaf1f16
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/java/org/apache/wayang/api/JavaApiTest.java
@@ -0,0 +1,711 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.api;
+
+import org.apache.wayang.core.api.WayangContext;
+import org.apache.wayang.core.util.WayangCollections;
+import org.apache.wayang.java.Java;
+//import org.apache.wayang.spark.Spark;
+//import org.apache.wayang.sqlite3.Sqlite3;
+//import org.apache.wayang.sqlite3.operators.Sqlite3TableSource;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Test suite for the Java API.
+ */
+public class JavaApiTest {
+
+//    private Configuration sqlite3Configuration;
+//
+//    @Before
+//    public void setUp() throws SQLException, IOException {
+//        // Generate test data.
+//        this.sqlite3Configuration = new Configuration();
+//        File sqlite3dbFile = File.createTempFile("wayang-sqlite3", "db");
+//        sqlite3dbFile.deleteOnExit();
+//        this.sqlite3Configuration.setProperty("wayang.sqlite3.jdbc.url", "jdbc:sqlite:" + sqlite3dbFile.getAbsolutePath());
+//        try (Connection connection = Sqlite3.platform().createDatabaseDescriptor(this.sqlite3Configuration).createJdbcConnection()) {
+//            Statement statement = connection.createStatement();
+//            statement.addBatch("DROP TABLE IF EXISTS customer;");
+//            statement.addBatch("CREATE TABLE customer (name TEXT, age INT);");
+//            statement.addBatch("INSERT INTO customer VALUES ('John', 20)");
+//            statement.addBatch("INSERT INTO customer VALUES ('Timmy', 16)");
+//            statement.addBatch("INSERT INTO customer VALUES ('Evelyn', 35)");
+//            statement.executeBatch();
+//        }
+//    }
+
+    @Test
+    public void testMapReduce() {
+        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
+
+        List<Integer> inputCollection = Arrays.asList(0, 1, 2, 3, 4);
+        Collection<Integer> outputCollection = javaPlanBuilder
+                .loadCollection(inputCollection).withName("load numbers")
+                .map(i -> i * i).withName("square")
+                .reduce((a, b) -> a + b).withName("sum")
+                .collect();
+
+        Assert.assertEquals(WayangCollections.asSet(1 + 4 + 9 + 16), WayangCollections.asSet(outputCollection));
+    }
+
+//    @Test
+//    public void testMapReduceBy() {
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
+//
+//        List<Integer> inputCollection = Arrays.asList(0, 1, 2, 3, 4);
+//        Collection<Integer> outputCollection = javaPlanBuilder
+//                .loadCollection(inputCollection).withName("load numbers")
+//                .map(i -> i * i).withName("square")
+//                .reduceByKey(i -> i & 1, (a, b) -> a + b).withName("sum")
+//                .collect();
+//
+//        Assert.assertEquals(WayangCollections.asSet(4 + 16, 1 + 9), WayangCollections.asSet(outputCollection));
+//    }
+//
+//    @Test
+//    public void testBroadcast2() {
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
+//
+//        List<Integer> inputCollection = Arrays.asList(0, 1, 2, 3, 4);
+//        List<Integer> offsetCollection = Collections.singletonList(-2);
+//
+//        LoadCollectionDataQuantaBuilder<Integer> offsetDataQuanta = javaPlanBuilder
+//                .loadCollection(offsetCollection)
+//                .withName("load offset");
+//
+//        Collection<Integer> outputCollection = javaPlanBuilder
+//                .loadCollection(inputCollection).withName("load numbers")
+//                .map(new AddOffset("offset")).withName("add offset").withBroadcast(offsetDataQuanta, "offset")
+//                .collect();
+//
+//        Assert.assertEquals(WayangCollections.asSet(-2, -1, 0, 1, 2), WayangCollections.asSet(outputCollection));
+//    }
+//
+//    @Test
+//    public void testCustomOperatorShortCut() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//
+//        final List<Integer> inputValues = WayangArrays.asList(0, 1, 2, 3);
+//
+//        // Build and execute a Wayang plan.
+//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
+//                .loadCollection(inputValues).withName("Load input values")
+//                .<Integer>customOperator(new JavaMapOperator<>(
+//                        DataSetType.createDefault(Integer.class),
+//                        DataSetType.createDefault(Integer.class),
+//                        new TransformationDescriptor<>(
+//                                i -> i + 2,
+//                                Integer.class, Integer.class
+//                        )
+//                )).withName("Add 2")
+//                .collect();
+//
+//        // Check the outcome.
+//        final List<Integer> expectedOutputValues = WayangArrays.asList(2, 3, 4, 5);
+//        Assert.assertEquals(WayangCollections.asSet(expectedOutputValues), WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testWordCount() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//
+//        final List<String> inputValues = Arrays.asList("Big data is big.", "Is data big data?");
+//
+//        // Build and execute a Wayang plan.
+//        final Collection<Tuple2<String, Integer>> outputValues = new JavaPlanBuilder(wayangContext)
+//                .loadCollection(inputValues).withName("Load input values")
+//                .flatMap(line -> Arrays.asList(line.split("\\s+"))).withName("Split words")
+//                .map(token -> token.replaceAll("\\W+", "").toLowerCase()).withName("To lower case")
+//                .map(word -> new Tuple2<>(word, 1)).withName("Attach counter")
+//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.field0, t1.field1 + t2.field1)).withName("Sum counters")
+//                .collect();
+//
+//        // Check the outcome.
+//        final Set<Tuple2<String, Integer>> expectedOutputValues = WayangCollections.asSet(
+//                new Tuple2<>("big", 3),
+//                new Tuple2<>("is", 2),
+//                new Tuple2<>("data", 3)
+//        );
+//        Assert.assertEquals(WayangCollections.asSet(expectedOutputValues), WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testWordCountOnSparkAndJava() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
+//
+//        final List<String> inputValues = Arrays.asList("Big data is big.", "Is data big data?");
+//
+//        // Build and execute a Wayang plan.
+//        final Collection<Tuple2<String, Integer>> outputValues = new JavaPlanBuilder(wayangContext)
+//                .loadCollection(inputValues).withName("Load input values")
+//                .flatMap(line -> Arrays.asList(line.split("\\s+"))).withName("Split words")
+//                .map(token -> token.replaceAll("\\W+", "").toLowerCase()).withName("To lower case")
+//                .map(word -> new Tuple2<>(word, 1)).withName("Attach counter")
+//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.field0, t1.field1 + t2.field1)).withName("Sum counters")
+//                .collect();
+//
+//        // Check the outcome.
+//        final Set<Tuple2<String, Integer>> expectedOutputValues = WayangCollections.asSet(
+//                new Tuple2<>("big", 3),
+//                new Tuple2<>("is", 2),
+//                new Tuple2<>("data", 3)
+//        );
+//        Assert.assertEquals(WayangCollections.asSet(expectedOutputValues), WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testSample() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
+//
+//        // Create some input values.
+//        final List<Integer> inputValues = WayangArrays.asList(WayangArrays.range(100));
+//
+//        // Build and execute a Wayang plan.
+//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
+//                .loadCollection(inputValues).withName("Load input values")
+//                .sample(10).withName("Sample")
+//                .collect();
+//
+//        // Check the outcome.
+//        Assert.assertEquals(10, outputValues.size());
+//        Assert.assertEquals(10, WayangCollections.asSet(outputValues).size());
+//
+//    }
+//
+//    @Test
+//    public void testDoWhile() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//
+//        // Generate test data.
+//        final List<Integer> inputValues = WayangArrays.asList(1, 2);
+//
+//        // Build and execute a word count WayangPlan.
+//
+//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
+//                .loadCollection(inputValues).withName("Load input values")
+//                .doWhile(
+//                        values -> values.stream().mapToInt(i -> i).sum() > 100,
+//                        start -> {
+//                            final GlobalReduceDataQuantaBuilder<Integer> sum =
+//                                    start.reduce((a, b) -> a + b).withName("sum");
+//                            return new Tuple<>(
+//                                    start.union(sum).withName("Old+new"),
+//                                    sum
+//                            );
+//                        }
+//                ).withConditionClass(Integer.class).withName("While <= 100")
+//                .collect();
+//
+//        Set<Integer> expectedValues = WayangCollections.asSet(1, 2, 3, 6, 12, 24, 48, 96, 192);
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    private static class AddOffset implements FunctionDescriptor.ExtendedSerializableFunction<Integer, Integer> {
+//
+//        private final String broadcastName;
+//
+//        private int offset;
+//
+//        public AddOffset(String broadcastName) {
+//            this.broadcastName = broadcastName;
+//        }
+//
+//        @Override
+//        public void open(ExecutionContext ctx) {
+//            this.offset = WayangCollections.getSingle(ctx.<Integer>getBroadcast(this.broadcastName));
+//        }
+//
+//        @Override
+//        public Integer apply(Integer input) {
+//            return input + this.offset;
+//        }
+//    }
+//
+//    @Test
+//    public void testRepeat() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//
+//        // Generate test data.
+//        final List<Integer> inputValues = WayangArrays.asList(1, 2);
+//
+//        // Build and execute a word count WayangPlan.
+//
+//        final Collection<Integer> outputValues = new JavaPlanBuilder(wayangContext)
+//                .loadCollection(inputValues).withName("Load input values")
+//                .repeat(3, start -> start
+//                        .reduce((a, b) -> a * b).withName("Multiply")
+//                        .flatMap(v -> Arrays.asList(v, v + 1)).withName("Duplicate").withOutputClass(Integer.class)
+//                ).withName("Repeat 3x")
+//                .collect();
+//
+//        Set<Integer> expectedValues = WayangCollections.asSet(42, 43);
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    private static class SelectWords implements PredicateDescriptor.ExtendedSerializablePredicate<String> {
+//
+//        private final String broadcastName;
+//
+//        private Collection<Character> selectors;
+//
+//        public SelectWords(String broadcastName) {
+//            this.broadcastName = broadcastName;
+//        }
+//
+//        @Override
+//        public void open(ExecutionContext ctx) {
+//            this.selectors = ctx.getBroadcast(this.broadcastName);
+//        }
+//
+//        @Override
+//        public boolean test(String word) {
+//            return this.selectors.stream().anyMatch(c -> word.indexOf(c) >= 0);
+//        }
+//    }
+//
+//    @Test
+//    public void testBroadcast() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<String> inputValues = Arrays.asList("Hello", "World", "Hi", "Mars");
+//        final List<Character> selectors = Arrays.asList('o', 'l');
+//
+//        // Execute the job.
+//        final DataQuantaBuilder<?, Character> selectorsDataSet = builder.loadCollection(selectors).withName("Load selectors");
+//        final Collection<String> outputValues = builder
+//                .loadCollection(inputValues).withName("Load input values")
+//                .filter(new SelectWords("selectors")).withName("Filter words")
+//                .withBroadcast(selectorsDataSet, "selectors")
+//                .collect();
+//
+//        // Verify the outcome.
+//        Set<String> expectedValues = WayangCollections.asSet("Hello", "World");
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testGroupBy() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Integer> inputValues = Arrays.asList(1, 2, 3, 4, 5, 7, 8, 9, 10);
+//
+//        // Execute the job.
+//        final Collection<Double> outputValues = builder
+//                .loadCollection(inputValues).withName("Load input values")
+//                .groupByKey(i -> i % 2).withName("group odd and even")
+//                .map(group -> {
+//                    List<Integer> sortedGroup = StreamSupport.stream(group.spliterator(), false)
+//                            .sorted()
+//                            .collect(Collectors.toList());
+//                    int sizeDivTwo = sortedGroup.size() / 2;
+//                    return sortedGroup.size() % 2 == 0 ?
+//                            (sortedGroup.get(sizeDivTwo - 1) + sortedGroup.get(sizeDivTwo)) / 2d :
+//                            (double) sortedGroup.get(sizeDivTwo);
+//                })
+//                .collect();
+//
+//        // Verify the outcome.
+//        Set<Double> expectedValues = WayangCollections.asSet(5d, 6d);
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testJoin() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
+//                new Tuple2<>("Water", 0),
+//                new Tuple2<>("Tonic", 5),
+//                new Tuple2<>("Juice", 10)
+//        );
+//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
+//                new Tuple2<>("Apple juice", "Juice"),
+//                new Tuple2<>("Tap water", "Water"),
+//                new Tuple2<>("Orange juice", "Juice")
+//        );
+//
+//        // Execute the job.
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
+//        final Collection<Tuple2<String, Integer>> outputValues = dataQuanta1
+//                .join(Tuple2::getField0, dataQuanta2, Tuple2::getField1)
+//                .map(joinTuple -> new Tuple2<>(joinTuple.getField1().getField0(), joinTuple.getField0().getField1()))
+//                .collect();
+//
+//        // Verify the outcome.
+//        Set<Tuple2<String, Integer>> expectedValues = WayangCollections.asSet(
+//                new Tuple2<>("Apple juice", 10),
+//                new Tuple2<>("Orange juice", 10),
+//                new Tuple2<>("Tap water", 0)
+//        );
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testJoinAndAssemble() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
+//                new Tuple2<>("Water", 0),
+//                new Tuple2<>("Tonic", 5),
+//                new Tuple2<>("Juice", 10)
+//        );
+//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
+//                new Tuple2<>("Apple juice", "Juice"),
+//                new Tuple2<>("Tap water", "Water"),
+//                new Tuple2<>("Orange juice", "Juice")
+//        );
+//
+//        // Execute the job.
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
+//        final Collection<Tuple2<String, Integer>> outputValues = dataQuanta1.keyBy(Tuple2::getField0)
+//                .join(dataQuanta2.keyBy(Tuple2::getField1))
+//                .assemble((val1, val2) -> new Tuple2<>(val2.getField0(), val1.getField1()))
+//                .collect();
+//
+//        // Verify the outcome.
+//        Set<Tuple2<String, Integer>> expectedValues = WayangCollections.asSet(
+//                new Tuple2<>("Apple juice", 10),
+//                new Tuple2<>("Orange juice", 10),
+//                new Tuple2<>("Tap water", 0)
+//        );
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testCoGroup() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
+//                new Tuple2<>("Water", 0),
+//                new Tuple2<>("Cola", 5),
+//                new Tuple2<>("Juice", 10)
+//        );
+//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
+//                new Tuple2<>("Apple juice", "Juice"),
+//                new Tuple2<>("Tap water", "Water"),
+//                new Tuple2<>("Orange juice", "Juice")
+//        );
+//
+//        // Execute the job.
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
+//        final Collection<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> outputValues = dataQuanta1
+//                .coGroup(Tuple2::getField0, dataQuanta2, Tuple2::getField1)
+//                .map(joinTuple -> new Tuple2<>(
+//                        WayangCollections.asSet(joinTuple.getField0()),
+//                        WayangCollections.asSet(joinTuple.getField1())
+//                ))
+//                .collect();
+//
+//        // Verify the outcome.
+//        Set<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> expectedValues = WayangCollections.asSet(
+//                new Tuple2<>(
+//                        WayangCollections.asSet(new Tuple2<>("Water", 0)),
+//                        WayangCollections.asSet(new Tuple2<>("Tap water", "Water"))
+//                ),
+//                new Tuple2<>(
+//                        WayangCollections.asSet(new Tuple2<>("Cola", 5)),
+//                        WayangCollections.asSet()
+//                ), new Tuple2<>(
+//                        WayangCollections.asSet(new Tuple2<>("Juice", 10)),
+//                        WayangCollections.asSet(new Tuple2<>("Apple juice", "Juice"), new Tuple2<>("Orange juice", "Juice"))
+//                )
+//        );
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testCoGroupViaKeyBy() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin()).with(Spark.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Tuple2<String, Integer>> inputValues1 = Arrays.asList(
+//                new Tuple2<>("Water", 0),
+//                new Tuple2<>("Cola", 5),
+//                new Tuple2<>("Juice", 10)
+//        );
+//        final List<Tuple2<String, String>> inputValues2 = Arrays.asList(
+//                new Tuple2<>("Apple juice", "Juice"),
+//                new Tuple2<>("Tap water", "Water"),
+//                new Tuple2<>("Orange juice", "Juice")
+//        );
+//
+//        // Execute the job.
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, Integer>> dataQuanta1 = builder.loadCollection(inputValues1);
+//        final LoadCollectionDataQuantaBuilder<Tuple2<String, String>> dataQuanta2 = builder.loadCollection(inputValues2);
+//        final Collection<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> outputValues =
+//                dataQuanta1.keyBy(Tuple2::getField0)
+//                        .coGroup(dataQuanta2.keyBy(Tuple2::getField1))
+//                        .map(joinTuple -> new Tuple2<>(
+//                                WayangCollections.asSet(joinTuple.getField0()),
+//                                WayangCollections.asSet(joinTuple.getField1())
+//                        ))
+//                        .collect();
+//
+//        // Verify the outcome.
+//        Set<Tuple2<Set<Tuple2<String, Integer>>, Set<Tuple2<String, String>>>> expectedValues = WayangCollections.asSet(
+//                new Tuple2<>(
+//                        WayangCollections.asSet(new Tuple2<>("Water", 0)),
+//                        WayangCollections.asSet(new Tuple2<>("Tap water", "Water"))
+//                ),
+//                new Tuple2<>(
+//                        WayangCollections.asSet(new Tuple2<>("Cola", 5)),
+//                        WayangCollections.asSet()
+//                ), new Tuple2<>(
+//                        WayangCollections.asSet(new Tuple2<>("Juice", 10)),
+//                        WayangCollections.asSet(new Tuple2<>("Apple juice", "Juice"), new Tuple2<>("Orange juice", "Juice"))
+//                )
+//        );
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testIntersect() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Integer> inputValues1 = Arrays.asList(1, 2, 3, 4, 5, 7, 8, 9, 10);
+//        final List<Integer> inputValues2 = Arrays.asList(0, 2, 3, 3, 4, 5, 7, 8, 9, 11);
+//
+//        // Execute the job.
+//        final LoadCollectionDataQuantaBuilder<Integer> dataQuanta1 = builder.loadCollection(inputValues1);
+//        final LoadCollectionDataQuantaBuilder<Integer> dataQuanta2 = builder.loadCollection(inputValues2);
+//        final Collection<Integer> outputValues = dataQuanta1.intersect(dataQuanta2).collect();
+//
+//        // Verify the outcome.
+//        Set<Integer> expectedValues = WayangCollections.asSet(2, 3, 4, 5, 7, 8, 9);
+//        Assert.assertEquals(expectedValues, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testSort() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        final List<Integer> inputValues1 = Arrays.asList(3, 4, 5, 2, 1);
+//
+//        // Execute the job.
+//        final LoadCollectionDataQuantaBuilder<Integer> dataQuanta1 = builder.loadCollection(inputValues1);
+//        final Collection<Integer> outputValues = dataQuanta1.sort(r -> r).collect();
+//
+//        // Verify the outcome.
+//        List<Integer> expectedValues = Arrays.asList(1, 2, 3, 4, 5);
+//        Assert.assertEquals(expectedValues, WayangCollections.asList(outputValues));
+//    }
+//
+//
+//    @Test
+//    public void testPageRank() {
+//        // Set up WayangContext.
+//        WayangContext wayangContext = new WayangContext()
+//                .with(Java.basicPlugin())
+//                .with(Java.graphPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Create a test graph.
+//        Collection<Tuple2<Long, Long>> edges = Arrays.asList(
+//                new Tuple2<>(0L, 1L),
+//                new Tuple2<>(0L, 2L),
+//                new Tuple2<>(0L, 3L),
+//                new Tuple2<>(1L, 0L),
+//                new Tuple2<>(2L, 1L),
+//                new Tuple2<>(3L, 2L),
+//                new Tuple2<>(3L, 1L)
+//        );
+//
+//        // Execute the job.
+//        Collection<Tuple2<Long, Float>> pageRanks = builder.loadCollection(edges).asEdges()
+//                .pageRank(20)
+//                .collect();
+//        List<Tuple2<Long, Float>> sortedPageRanks = new ArrayList<>(pageRanks);
+//        sortedPageRanks.sort((pr1, pr2) -> Float.compare(pr2.field1, pr1.field1));
+//
+//        System.out.println(sortedPageRanks);
+//        Assert.assertEquals(1L, sortedPageRanks.get(0).field0.longValue());
+//        Assert.assertEquals(0L, sortedPageRanks.get(1).field0.longValue());
+//        Assert.assertEquals(2L, sortedPageRanks.get(2).field0.longValue());
+//        Assert.assertEquals(3L, sortedPageRanks.get(3).field0.longValue());
+//    }
+//
+//    @Test
+//    public void testMapPartitions() {
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        List<Integer> inputValues = WayangArrays.asList(0, 1, 2, 3, 4, 6, 8);
+//
+//        // Execute the job.
+//        Collection<Tuple2<String, Integer>> outputValues = builder.loadCollection(inputValues)
+//                .mapPartitions(partition -> {
+//                    int numEvens = 0, numOdds = 0;
+//                    for (Integer value : partition) {
+//                        if ((value & 1) == 0) numEvens++;
+//                        else numOdds++;
+//                    }
+//                    return Arrays.asList(
+//                            new Tuple2<>("odd", numOdds),
+//                            new Tuple2<>("even", numEvens)
+//                    );
+//                })
+//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.getField0(), t1.getField1() + t2.getField1()))
+//                .collect();
+//
+//        // Check the output.
+//        Set<Tuple2<String, Integer>> expectedOutput = WayangCollections.asSet(
+//                new Tuple2<>("even", 5), new Tuple2<>("odd", 2)
+//        );
+//        Assert.assertEquals(expectedOutput, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testZipWithId() {
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        List<Integer> inputValues = new ArrayList<>(42 * 100);
+//        for (int i = 0; i < 100; i++) {
+//            for (int j = 0; j < 42; j++) {
+//                inputValues.add(i);
+//            }
+//        }
+//
+//        // Execute the job.
+//        Collection<Tuple2<Integer, Integer>> outputValues = builder.loadCollection(inputValues)
+//                .zipWithId()
+//                .groupByKey(Tuple2::getField1)
+//                .map(group -> {
+//                    int distinctIds = (int) StreamSupport.stream(group.spliterator(), false)
+//                            .map(Tuple2::getField0)
+//                            .distinct()
+//                            .count();
+//                    return new Tuple2<>(distinctIds, 1);
+//                })
+//                .reduceByKey(Tuple2::getField0, (t1, t2) -> new Tuple2<>(t1.getField0(), t1.getField1() + t2.getField1()))
+//                .collect();
+//
+//        // Check the output.
+//        Set<Tuple2<Integer, Integer>> expectedOutput = Collections.singleton(new Tuple2<>(42, 100));
+//        Assert.assertEquals(expectedOutput, WayangCollections.asSet(outputValues));
+//    }
+//
+//    @Test
+//    public void testWriteTextFile() throws IOException, URISyntaxException {
+//        WayangContext wayangContext = new WayangContext().with(Java.basicPlugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangContext);
+//
+//        // Generate test data.
+//        List<Double> inputValues = Arrays.asList(0d, 1 / 3d, 2 / 3d, 1d, 4 / 3d, 5 / 3d);
+//
+//        // Execute the job.
+//        File tempDir = LocalFileSystem.findTempDir();
+//        String targetUrl = LocalFileSystem.toURL(new File(tempDir, "testWriteTextFile.txt"));
+//
+//        builder
+//                .loadCollection(inputValues)
+//                .writeTextFile(targetUrl, d -> String.format("%.2f", d), "testWriteTextFile()");
+//
+//        // Check the output.
+//        Set<String> actualLines = Files.lines(Paths.get(new URI(targetUrl))).collect(Collectors.toSet());
+//        Set<String> expectedLines = inputValues.stream().map(d -> String.format("%.2f", d)).collect(Collectors.toSet());
+//        Assert.assertEquals(expectedLines, actualLines);
+//    }
+//
+//    @Test
+//    public void testSqlOnJava() throws IOException, SQLException {
+//        // Execute job.
+//        final WayangContext wayangCtx = new WayangContext(this.sqlite3Configuration)
+//                .with(Java.basicPlugin())
+//                .with(Sqlite3.plugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangCtx, "testSqlOnJava()");
+//        final Collection<String> outputValues = builder
+//                .readTable(new Sqlite3TableSource("customer", "name", "age"))
+//                .filter(r -> (Integer) r.getField(1) >= 18).withSqlUdf("age >= 18").withTargetPlatform(Java.platform())
+//                .asRecords().projectRecords(new String[]{"name"})
+//                .map(record -> (String) record.getField(0))
+//                .collect();
+//
+//        // Test the outcome.
+//        Assert.assertEquals(
+//                WayangCollections.asSet("John", "Evelyn"),
+//                WayangCollections.asSet(outputValues)
+//        );
+//    }
+//
+//    @Test
+//    public void testSqlOnSqlite3() throws IOException, SQLException {
+//        // Execute job.
+//        final WayangContext wayangCtx = new WayangContext(this.sqlite3Configuration)
+//                .with(Java.basicPlugin())
+//                .with(Sqlite3.plugin());
+//        JavaPlanBuilder builder = new JavaPlanBuilder(wayangCtx, "testSqlOnSqlite3()");
+//        final Collection<String> outputValues = builder
+//                .readTable(new Sqlite3TableSource("customer", "name", "age"))
+//                .filter(r -> (Integer) r.getField(1) >= 18).withSqlUdf("age >= 18")
+//                .asRecords().projectRecords(new String[]{"name"}).withTargetPlatform(Sqlite3.platform())
+//                .map(record -> (String) record.getField(0))
+//                .collect();
+//
+//        // Test the outcome.
+//        Assert.assertEquals(
+//                WayangCollections.asSet("John", "Evelyn"),
+//                WayangCollections.asSet(outputValues)
+//        );
+//    }
+
+}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/api/ApiTest.scala b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/api/ApiTest.scala
new file mode 100644
index 0000000..ed1a7ac
--- /dev/null
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-api/src/test/scala/org/apache/wayang/api/ApiTest.scala
@@ -0,0 +1,575 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.api
+
+import org.apache.wayang.basic.WayangBasics
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializablePredicate
+import org.apache.wayang.core.function.{ExecutionContext, TransformationDescriptor}
+import org.apache.wayang.core.util.fs.LocalFileSystem
+import org.apache.wayang.java.Java
+import org.junit.{Assert, Test}
+
+import java.io.File
+import java.net.URI
+import java.nio.file.{Files, Paths}
+import java.sql.{Connection, Statement}
+import java.util.function.Consumer
+
+/**
+  * Tests the Wayang API.
+  */
+class ApiTest {
+
+  @Test
+  def testReadMapCollect(): Unit = {
+    // Set up WayangContext.
+    val wayangContext = new WayangContext()
+                         .withPlugin(Java.basicPlugin)
+//                         .withPlugin(Spark.basicPlugin)
+    // Generate some test data.
+    val inputValues = (for (i <- 1 to 10) yield i).toArray
+
+    // Build and execute a Wayang plan.
+    val outputValues = wayangContext
+      .loadCollection(inputValues).withName("Load input values")
+      .map(_ + 2).withName("Add 2")
+      .collect()
+
+    // Check the outcome.
+    val expectedOutputValues = inputValues.map(_ + 2)
+    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
+  }
+
+//  @Test
+//  def testCustomOperator(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = (for (i <- 1 to 10) yield i).toArray
+//
+//    // Build and execute a Wayang plan.
+//    val inputDataSet = wayang.loadCollection(inputValues).withName("Load input values")
+//
+//    // Add the custom operator.
+//    val IndexedSeq(addedValues) = wayang.customOperator(new JavaMapOperator(
+//      dataSetType[Int],
+//      dataSetType[Int],
+//      new TransformationDescriptor(
+//        toSerializableFunction[Int, Int](_ + 2),
+//        basicDataUnitType[Int], basicDataUnitType[Int]
+//      )
+//    ), inputDataSet)
+//    addedValues.withName("Add 2")
+//
+//    // Collect the result.
+//    val outputValues = addedValues.asInstanceOf[DataQuanta[Int]].collect()
+//
+//    // Check the outcome.
+//    val expectedOutputValues = inputValues.map(_ + 2)
+//    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
+//  }
+//
+//  @Test
+//  def testCustomOperatorShortCut(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = (for (i <- 1 to 10) yield i).toArray
+//
+//    // Build and execute a Wayang plan.
+//    val outputValues = wayang
+//      .loadCollection(inputValues).withName("Load input values")
+//      .customOperator[Int](new JavaMapOperator(
+//      dataSetType[Int],
+//      dataSetType[Int],
+//      new TransformationDescriptor(
+//        toSerializableFunction[Int, Int](_ + 2),
+//        basicDataUnitType[Int], basicDataUnitType[Int]
+//      )
+//    )).withName("Add 2")
+//      .collect()
+//
+//    // Check the outcome.
+//    val expectedOutputValues = inputValues.map(_ + 2)
+//    Assert.assertArrayEquals(expectedOutputValues, outputValues.toArray)
+//  }
+//
+//  @Test
+//  def testWordCount(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = Array("Big data is big.", "Is data big data?")
+//
+//    // Build and execute a word count WayangPlan.
+//    val wordCounts = wayang
+//      .loadCollection(inputValues).withName("Load input values")
+//      .flatMap(_.split("\\s+")).withName("Split words")
+//      .map(_.replaceAll("\\W+", "").toLowerCase).withName("To lowercase")
+//      .map((_, 1)).withName("Attach counter")
+//      .reduceByKey(_._1, (a, b) => (a._1, a._2 + b._2)).withName("Sum counters")
+//      .collect().toSet
+//
+//    val expectedWordCounts = Set(("big", 3), ("is", 2), ("data", 3))
+//
+//    Assert.assertEquals(expectedWordCounts, wordCounts)
+//  }
+//
+//  @Test
+//  def testWordCountOnSparkAndJava(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = Array("Big data is big.", "Is data big data?")
+//
+//    // Build and execute a word count WayangPlan.
+//    val wordCounts = wayang
+//      .loadCollection(inputValues).withName("Load input values").withTargetPlatforms(Java.platform)
+//      .flatMap(_.split("\\s+")).withName("Split words").withTargetPlatforms(Java.platform)
+//      .map(_.replaceAll("\\W+", "").toLowerCase).withName("To lowercase").withTargetPlatforms(Spark.platform)
+//      .map((_, 1)).withName("Attach counter").withTargetPlatforms(Spark.platform)
+//      .reduceByKey(_._1, (a, b) => (a._1, a._2 + b._2)).withName("Sum counters").withTargetPlatforms(Spark.platform)
+//      .collect().toSet
+//
+//    val expectedWordCounts = Set(("big", 3), ("is", 2), ("data", 3))
+//
+//    Assert.assertEquals(expectedWordCounts, wordCounts)
+//  }
+//
+//  @Test
+//  def testSample(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = for (i <- 0 until 100) yield i
+//
+//    // Build and execute the WayangPlan.
+//    val sample = wayang
+//      .loadCollection(inputValues)
+//      .sample(10)
+//      .collect()
+//
+//    // Check the result.
+//    Assert.assertEquals(10, sample.size)
+//    Assert.assertEquals(10, sample.toSet.size)
+//  }
+//
+//  @Test
+//  def testDoWhile(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = Array(1, 2)
+//
+//    // Build and execute a word count WayangPlan.
+//
+//    val values = wayang
+//      .loadCollection(inputValues).withName("Load input values")
+//      .doWhile[Int](vals => vals.max > 100, {
+//      start =>
+//        val sum = start.reduce(_ + _).withName("Sum")
+//        (start.union(sum).withName("Old+new"), sum)
+//    }).withName("While <= 100")
+//      .collect().toSet
+//
+//    val expectedValues = Set(1, 2, 3, 6, 12, 24, 48, 96, 192)
+//    Assert.assertEquals(expectedValues, values)
+//  }
+//
+//  @Test
+//  def testRepeat(): Unit = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    // Generate some test data.
+//    val inputValues = Array(1, 2)
+//
+//    // Build and execute a word count WayangPlan.
+//
+//    val values = wayang
+//      .loadCollection(inputValues).withName("Load input values").withName(inputValues.mkString(","))
+//      .repeat(3,
+//        _.reduce(_ * _).withName("Multiply")
+//          .flatMap(v => Seq(v, v + 1)).withName("Duplicate")
+//      ).withName("Repeat 3x")
+//      .collect().toSet
+//
+//    // initial: 1,2 -> 1st: 2,3 -> 2nd: 6,7 => 3rd: 42,43
+//    val expectedValues = Set(42, 43)
+//    Assert.assertEquals(expectedValues, values)
+//  }
+//
+//  @Test
+//  def testBroadcast() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//    val builder = new PlanBuilder(wayang)
+//
+//    // Generate some test data.
+//    val inputStrings = Array("Hello", "World", "Hi", "Mars")
+//    val selectors = Array('o', 'l')
+//
+//    val selectorsDataSet = builder.loadCollection(selectors).withName("Load selectors")
+//
+//    // Build and execute a word count WayangPlan.
+//    val values = builder
+//      .loadCollection(inputStrings).withName("Load input values")
+//      .filterJava(new ExtendedSerializablePredicate[String] {
+//
+//        var selectors: Iterable[Char] = _
+//
+//        override def open(ctx: ExecutionContext): Unit = {
+//          import scala.collection.JavaConversions._
+//          selectors = collectionAsScalaIterable(ctx.getBroadcast[Char]("selectors"))
+//        }
+//
+//        override def test(t: String): Boolean = selectors.forall(selector => t.contains(selector))
+//
+//      }).withName("Filter words")
+//      .withBroadcast(selectorsDataSet, "selectors")
+//      .collect().toSet
+//
+//    val expectedValues = Set("Hello", "World")
+//    Assert.assertEquals(expectedValues, values)
+//  }
+//
+//  @Test
+//  def testGroupBy() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues = Array(1, 2, 3, 4, 5, 7, 8, 9, 10)
+//
+//    val result = wayang
+//      .loadCollection(inputValues)
+//      .groupByKey(_ % 2).withName("group odd and even")
+//      .map {
+//        group =>
+//          import scala.collection.JavaConversions._
+//          val buffer = group.toBuffer
+//          buffer.sortBy(identity)
+//          if (buffer.size % 2 == 0) (buffer(buffer.size / 2 - 1) + buffer(buffer.size / 2)) / 2
+//          else buffer(buffer.size / 2)
+//      }.withName("median")
+//      .collect()
+//
+//    val expectedValues = Set(5, 6)
+//    Assert.assertEquals(expectedValues, result.toSet)
+//  }
+//
+//  @Test
+//  def testGroup() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues = Array(1, 2, 3, 4, 5, 7, 8, 9, 10)
+//
+//    val result = wayang
+//      .loadCollection(inputValues)
+//      .group()
+//      .map {
+//        group =>
+//          import scala.collection.JavaConversions._
+//          val buffer = group.toBuffer
+//          buffer.sortBy(int => int)
+//          if (buffer.size % 2 == 0) (buffer(buffer.size / 2) + buffer(buffer.size / 2 + 1)) / 2
+//          else buffer(buffer.size / 2)
+//      }
+//      .collect()
+//
+//    val expectedValues = Set(5)
+//    Assert.assertEquals(expectedValues, result.toSet)
+//  }
+//
+//  @Test
+//  def testJoin() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues1 = Array(("Water", 0), ("Tonic", 5), ("Juice", 10))
+//    val inputValues2 = Array(("Apple juice", "Juice"), ("Tap water", "Water"), ("Orange juice", "Juice"))
+//
+//    val builder = new PlanBuilder(wayang)
+//    val dataQuanta1 = builder.loadCollection(inputValues1)
+//    val dataQuanta2 = builder.loadCollection(inputValues2)
+//    val result = dataQuanta1
+//      .join[(String, String), String](_._1, dataQuanta2, _._2)
+//      .map(joinTuple => (joinTuple.field1._1, joinTuple.field0._2))
+//      .collect()
+//
+//    val expectedValues = Set(("Apple juice", 10), ("Tap water", 0), ("Orange juice", 10))
+//    Assert.assertEquals(expectedValues, result.toSet)
+//  }
+//
+//  @Test
+//  def testJoinAndAssemble() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues1 = Array(("Water", 0), ("Tonic", 5), ("Juice", 10))
+//    val inputValues2 = Array(("Apple juice", "Juice"), ("Tap water", "Water"), ("Orange juice", "Juice"))
+//
+//    val builder = new PlanBuilder(wayang)
+//    val dataQuanta1 = builder.loadCollection(inputValues1)
+//    val dataQuanta2 = builder.loadCollection(inputValues2)
+//    val result = dataQuanta1.keyBy(_._1).join(dataQuanta2.keyBy(_._2))
+//      .assemble((dq1, dq2) => (dq2._1, dq1._2))
+//      .collect()
+//
+//    val expectedValues = Set(("Apple juice", 10), ("Tap water", 0), ("Orange juice", 10))
+//    Assert.assertEquals(expectedValues, result.toSet)
+//  }
+//
+//
+//  @Test
+//  def testCoGroup() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues1 = Array(("Water", 0), ("Cola", 5), ("Juice", 10))
+//    val inputValues2 = Array(("Apple juice", "Juice"), ("Tap water", "Water"), ("Orange juice", "Juice"))
+//
+//    val builder = new PlanBuilder(wayang)
+//    val dataQuanta1 = builder.loadCollection(inputValues1)
+//    val dataQuanta2 = builder.loadCollection(inputValues2)
+//    val result = dataQuanta1
+//      .coGroup[(String, String), String](_._1, dataQuanta2, _._2)
+//      .collect()
+//
+//    import scala.collection.JavaConversions._
+//    val actualValues = result.map(coGroup => (coGroup.field0.toSet, coGroup.field1.toSet)).toSet
+//    val expectedValues = Set(
+//      (Set(("Water", 0)), Set(("Tap water", "Water"))),
+//      (Set(("Cola", 5)), Set()),
+//      (Set(("Juice", 10)), Set(("Apple juice", "Juice"), ("Orange juice", "Juice")))
+//    )
+//    Assert.assertEquals(expectedValues, actualValues)
+//  }
+//
+//  @Test
+//  def testIntersect() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues1 = Array(1, 2, 3, 4, 5, 7, 8, 9, 10)
+//    val inputValues2 = Array(0, 2, 3, 3, 4, 5, 7, 8, 9, 11)
+//
+//    val builder = new PlanBuilder(wayang)
+//    val dataQuanta1 = builder.loadCollection(inputValues1)
+//    val dataQuanta2 = builder.loadCollection(inputValues2)
+//    val result = dataQuanta1
+//      .intersect(dataQuanta2)
+//      .collect()
+//
+//    val expectedValues = Set(2, 3, 4, 5, 7, 8, 9)
+//    Assert.assertEquals(expectedValues, result.toSet)
+//  }
+//
+//
+//  @Test
+//  def testSort() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues1 = Array(3, 4, 5, 2, 1)
+//
+//    val builder = new PlanBuilder(wayang)
+//    val dataQuanta1 = builder.loadCollection(inputValues1)
+//    val result = dataQuanta1
+//      .sort(r=>r)
+//      .collect()
+//
+//    val expectedValues = Array(1, 2, 3, 4, 5)
+//    Assert.assertArrayEquals(expectedValues, result.toArray)
+//  }
+//
+//
+//  @Test
+//  def testPageRank() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext()
+//      .withPlugin(Java.graphPlugin)
+//      .withPlugin(WayangBasics.graphPlugin)
+//      .withPlugin(Java.basicPlugin)
+//    import org.apache.wayang.api.graph._
+//
+//    val edges = Seq((0, 1), (0, 2), (0, 3), (1, 0), (2, 1), (3, 2), (3, 1)).map(t => Edge(t._1, t._2))
+//
+//    val pageRanks = wayang
+//      .loadCollection(edges).withName("Load edges")
+//      .pageRank(20).withName("PageRank")
+//      .collect()
+//      .map(t => t.field0.longValue -> t.field1)
+//      .toMap
+//
+//    print(pageRanks)
+//    // Let's not check absolute numbers but only the relative ordering.
+//    Assert.assertTrue(pageRanks(1) > pageRanks(0))
+//    Assert.assertTrue(pageRanks(0) > pageRanks(2))
+//    Assert.assertTrue(pageRanks(2) > pageRanks(3))
+//  }
+//
+//  @Test
+//  def testMapPartitions() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext()
+//      .withPlugin(Java.basicPlugin())
+//      .withPlugin(Spark.basicPlugin)
+//
+//    val typeCounts = wayang
+//      .loadCollection(Seq(0, 1, 2, 3, 4, 6, 8))
+//        .mapPartitions { ints =>
+//          var (numOdds, numEvens) = (0, 0)
+//          ints.foreach(i => if ((i & 1) == 0) numEvens += 1 else numOdds += 1)
+//          Seq(("odd", numOdds), ("even", numEvens))
+//        }
+//      .reduceByKey(_._1, { case ((kind1, count1), (kind2, count2)) => (kind1, count1 + count2) })
+//      .collect()
+//
+//    Assert.assertEquals(Set(("odd", 2), ("even", 5)), typeCounts.toSet)
+//  }
+//
+//  @Test
+//  def testZipWithId() = {
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin).withPlugin(Spark.basicPlugin)
+//
+//    val inputValues = for (i <- 0 until 100; j <- 0 until 42) yield i
+//
+//    val result = wayang
+//      .loadCollection(inputValues)
+//      .zipWithId
+//      .groupByKey(_.field1)
+//      .map { group =>
+//        import scala.collection.JavaConversions._
+//        (group.map(_.field0).toSet.size, 1)
+//      }
+//      .reduceByKey(_._1, (t1, t2) => (t1._1, t1._2 + t2._2))
+//      .collect()
+//
+//    val expectedValues = Set((42, 100))
+//    Assert.assertEquals(expectedValues, result.toSet)
+//  }
+//
+//  @Test
+//  def testWriteTextFile() = {
+//    val tempDir = LocalFileSystem.findTempDir
+//    val targetUrl = LocalFileSystem.toURL(new File(tempDir, "testWriteTextFile.txt"))
+//
+//    // Set up WayangContext.
+//    val wayang = new WayangContext().withPlugin(Java.basicPlugin)
+//
+//    val inputValues = for (i <- 0 to 5) yield i * 0.333333333333
+//
+//    val result = wayang
+//      .loadCollection(inputValues)
+//      .writeTextFile(targetUrl, formatterUdf = d => f"${d % .2f}")
+//
+//    val lines = scala.collection.mutable.Set[String]()
+//    Files.lines(Paths.get(new URI(targetUrl))).forEach(new Consumer[String] {
+//      override def accept(line: String): Unit = lines += line
+//    })
+//
+//    val expectedLines = inputValues.map(v => f"${v % .2f}").toSet
+//    Assert.assertEquals(expectedLines, lines)
+//  }
+//
+//  @Test
+//  def testSqlOnJava() = {
+//    // Initialize some test data.
+//    val configuration = new Configuration
+//    val sqlite3dbFile = File.createTempFile("wayang-sqlite3", "db")
+//    sqlite3dbFile.deleteOnExit()
+//    configuration.setProperty("wayang.sqlite3.jdbc.url", "jdbc:sqlite:" + sqlite3dbFile.getAbsolutePath)
+//
+//    try {
+//      val connection: Connection = Sqlite3.platform.createDatabaseDescriptor(configuration).createJdbcConnection
+//      try {
+//        val statement: Statement = connection.createStatement
+//        statement.addBatch("DROP TABLE IF EXISTS customer;")
+//        statement.addBatch("CREATE TABLE customer (name TEXT, age INT);")
+//        statement.addBatch("INSERT INTO customer VALUES ('John', 20)")
+//        statement.addBatch("INSERT INTO customer VALUES ('Timmy', 16)")
+//        statement.addBatch("INSERT INTO customer VALUES ('Evelyn', 35)")
+//        statement.executeBatch()
+//      } finally {
+//        if (connection != null) connection.close()
+//      }
+//    }
+//
+//    // Set up WayangContext.
+//    val wayang = new WayangContext(configuration).withPlugin(Java.basicPlugin).withPlugin(Sqlite3.plugin)
+//
+//    val result = wayang
+//      .readTable(new Sqlite3TableSource("customer", "name", "age"))
+//      .filter(r => r.getField(1).asInstanceOf[Integer] >= 18, sqlUdf = "age >= 18").withTargetPlatforms(Java.platform)
+//      .projectRecords(Seq("name"))
+//      .map(_.getField(0).asInstanceOf[String])
+//      .collect()
+//      .toSet
+//
+//    val expectedValues = Set("John", "Evelyn")
+//    Assert.assertEquals(expectedValues, result)
+//  }
+//
+//  @Test
+//  def testSqlOnSqlite3() = {
+//    // Initialize some test data.
+//    val configuration = new Configuration
+//    val sqlite3dbFile = File.createTempFile("wayang-sqlite3", "db")
+//    sqlite3dbFile.deleteOnExit()
+//    configuration.setProperty("wayang.sqlite3.jdbc.url", "jdbc:sqlite:" + sqlite3dbFile.getAbsolutePath)
+//
+//    try {
+//      val connection: Connection = Sqlite3.platform.createDatabaseDescriptor(configuration).createJdbcConnection
+//      try {
+//        val statement: Statement = connection.createStatement
+//        statement.addBatch("DROP TABLE IF EXISTS customer;")
+//        statement.addBatch("CREATE TABLE customer (name TEXT, age INT);")
+//        statement.addBatch("INSERT INTO customer VALUES ('John', 20)")
+//        statement.addBatch("INSERT INTO customer VALUES ('Timmy', 16)")
+//        statement.addBatch("INSERT INTO customer VALUES ('Evelyn', 35)")
+//        statement.executeBatch
+//      } finally {
+//        if (connection != null) connection.close()
+//      }
+//    }
+//
+//    // Set up WayangContext.
+//    val wayang = new WayangContext(configuration).withPlugin(Java.basicPlugin).withPlugin(Sqlite3.plugin)
+//
+//    val result = wayang
+//      .readTable(new Sqlite3TableSource("customer", "name", "age"))
+//      .filter(r => r.getField(1).asInstanceOf[Integer] >= 18, sqlUdf = "age >= 18")
+//      .projectRecords(Seq("name")).withTargetPlatforms(Sqlite3.platform)
+//      .map(_.getField(0).asInstanceOf[String])
+//      .collect()
+//      .toSet
+//
+//    val expectedValues = Set("John", "Evelyn")
+//    Assert.assertEquals(expectedValues, result)
+//  }
+}
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
index 48002de..bb171ef 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tagger/HackitTagger.java
@@ -32,6 +32,7 @@ import java.util.List;
  * tagging step in Hackit, this logic have and pre and post processing and they are acting like
  * template that follow same behaivor in every tagger
  */
+//TODO add the option of add a custom function
 public class HackitTagger implements Serializable {
 
     /**
diff --git a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java
index 3b55a51..92384f2 100644
--- a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java
+++ b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/RegressionIT.java
@@ -21,8 +21,8 @@ package org.apache.wayang.tests;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.wayang.api.JavaPlanBuilder;
-import org.apache.wayang.api.dataquantabuilder.LoadCollectionDataQuantaBuilder;
-import org.apache.wayang.api.dataquantabuilder.MapDataQuantaBuilder;
+import org.apache.wayang.api.dataquanta.builder.LoadCollectionDataQuantaBuilder;
+import org.apache.wayang.api.dataquanta.builder.MapDataQuantaBuilder;
 import org.apache.wayang.core.api.WayangContext;
 import org.apache.wayang.core.util.WayangArrays;
 import org.apache.wayang.java.Java;
diff --git a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java
index 3062afb..88fa42f 100644
--- a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java
+++ b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java
@@ -18,7 +18,7 @@
 
 package org.apache.wayang.tests;
 
-import org.apache.wayang.api.DataQuantaBuilder;
+import org.apache.wayang.api.dataquanta.DataQuantaBuilder;
 import org.apache.wayang.api.JavaPlanBuilder;
 import org.apache.wayang.basic.data.Record;
 import org.apache.wayang.basic.data.Tuple2;

[incubator-wayang] 08/15: [WAYANG-31] DataQuanta was refactored as abstract class

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 26af4622521a397fa3ef4a6a680d1ee8d0568a95
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Thu May 13 22:31:37 2021 -0400

    [WAYANG-31] DataQuanta was refactored as abstract class
---
 .../apache/wayang/api/dataquanta/DataQuanta.scala  | 263 ++++---------
 .../wayang/api/dataquanta/DataQuantaDefault.scala  | 409 +++++++++++++++++++++
 .../wayang/api/dataquanta/DataQuantaFactory.scala  |  49 +++
 .../wayang/api/dataquanta/JoinedDataQuanta.scala   |   2 +-
 .../wayang/api/dataquanta/KeyedDataQuanta.scala    |   7 +-
 .../main/scala/org/apache/wayang/api/package.scala |  10 +-
 6 files changed, 536 insertions(+), 204 deletions(-)

diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
index a614467..6e464e0 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuanta.scala
@@ -21,7 +21,7 @@ package org.apache.wayang.api.dataquanta
 
 import de.hpi.isg.profiledb.store.model.Experiment
 import org.apache.commons.lang3.Validate
-import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, groupedDataSetType, groupedDataUnitType, toConsumer, toSerializableBinaryOperator, toSerializableFlatteningFunction, toSerializableFunction, toSerializablePartitionFunction, toSerializablePredicate}
+import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, toConsumer, toSerializableBinaryOperator, toSerializableFlatteningFunction, toSerializableFunction, toSerializablePartitionFunction, toSerializablePredicate}
 import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
 import org.apache.wayang.basic.function.ProjectionDescriptor
 import org.apache.wayang.basic.operators._
@@ -35,7 +35,7 @@ import org.apache.wayang.core.platform.Platform
 import org.apache.wayang.core.util.{Tuple => WayangTuple}
 
 import java.lang.{Iterable => JavaIterable}
-import java.util.function.{Consumer, IntUnaryOperator, BiFunction => JavaBiFunction, Function => JavaFunction}
+import java.util.function.{Consumer, IntUnaryOperator, Function => JavaFunction}
 import java.util.{Collection => JavaCollection}
 import scala.collection.JavaConversions
 import scala.collection.JavaConversions._
@@ -43,12 +43,13 @@ import scala.reflect._
 
 /**
   * Represents an intermediate result/data flow edge in a [[WayangPlan]].
+  * However this is just a template that help to be easy extendable the API
   *
   * @param operator    a unary [[Operator]] that produces this instance
   * @param ev$1        the data type of the elements in this instance
   * @param planBuilder keeps track of the [[WayangPlan]] being build
   */
-class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: Int = 0)(implicit val planBuilder: PlanBuilder) {
+abstract class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: Int = 0)(implicit val planBuilder: PlanBuilder) {
 
   Validate.isTrue(operator.getNumOutputs > outputIndex)
 
@@ -67,8 +68,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @return a new instance representing the [[MapOperator]]'s output
     */
   def map[NewOut: ClassTag](udf: Out => NewOut,
-                            udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] =
+                            udfLoad: LoadProfileEstimator = null):  DataQuanta[NewOut] = {
     mapJava(toSerializableFunction(udf), udfLoad)
+  }
 
   /**
     * Feed this instance into a [[MapOperator]].
@@ -78,13 +80,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @return a new instance representing the [[MapOperator]]'s output
     */
   def mapJava[NewOut: ClassTag](udf: SerializableFunction[Out, NewOut],
-                                udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] = {
-    val mapOperator = new MapOperator(new TransformationDescriptor(
-      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], udfLoad
-    ))
-    this.connectTo(mapOperator, 0)
-    mapOperator
-  }
+                                udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut]
 
   /**
     * Feed this instance into a [[MapPartitionsOperator]].
@@ -96,8 +92,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     */
   def mapPartitions[NewOut: ClassTag](udf: Iterable[Out] => Iterable[NewOut],
                                       selectivity: ProbabilisticDoubleInterval = null,
-                                      udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] =
+                                      udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] = {
     mapPartitionsJava(toSerializablePartitionFunction(udf), selectivity, udfLoad)
+  }
 
   /**
     * Feed this instance into a [[MapPartitionsOperator]].
@@ -109,13 +106,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     */
   def mapPartitionsJava[NewOut: ClassTag](udf: SerializableFunction[JavaIterable[Out], JavaIterable[NewOut]],
                                           selectivity: ProbabilisticDoubleInterval = null,
-                                          udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] = {
-    val mapOperator = new MapPartitionsOperator(
-      new MapPartitionsDescriptor(udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad)
-    )
-    this.connectTo(mapOperator, 0)
-    mapOperator
-  }
+                                          udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut]
 
   /**
     * Feed this instance into a [[MapOperator]] with a [[ProjectionDescriptor]].
@@ -123,13 +114,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param fieldNames names of the fields to be projected
     * @return a new instance representing the [[MapOperator]]'s output
     */
-  def project[NewOut: ClassTag](fieldNames: Seq[String]): DataQuanta[NewOut] = {
-    val projectionOperator = new MapOperator(
-      new ProjectionDescriptor(basicDataUnitType[Out], basicDataUnitType[NewOut], fieldNames: _*)
-    )
-    this.connectTo(projectionOperator, 0)
-    projectionOperator
-  }
+  def project[NewOut: ClassTag](fieldNames: Seq[String]): DataQuanta[NewOut]
 
   /**
     * Connects the [[operator]] to a further [[Operator]].
@@ -153,8 +138,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
   def filter(udf: Out => Boolean,
              sqlUdf: String = null,
              selectivity: ProbabilisticDoubleInterval = null,
-             udfLoad: LoadProfileEstimator = null) =
+             udfLoad: LoadProfileEstimator = null) = {
     filterJava(toSerializablePredicate(udf), sqlUdf, selectivity, udfLoad)
+  }
 
   /**
     * Feed this instance into a [[FilterOperator]].
@@ -168,13 +154,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
   def filterJava(udf: SerializablePredicate[Out],
                  sqlUdf: String = null,
                  selectivity: ProbabilisticDoubleInterval = null,
-                 udfLoad: LoadProfileEstimator = null): DataQuanta[Out] = {
-    val filterOperator = new FilterOperator(new PredicateDescriptor(
-      udf, this.output.getType.getDataUnitType.toBasicDataUnitType, selectivity, udfLoad
-    ).withSqlImplementation(sqlUdf))
-    this.connectTo(filterOperator, 0)
-    filterOperator
-  }
+                 udfLoad: LoadProfileEstimator = null): DataQuanta[Out]
 
   /**
     * Feed this instance into a [[FlatMapOperator]].
@@ -186,8 +166,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     */
   def flatMap[NewOut: ClassTag](udf: Out => Iterable[NewOut],
                                 selectivity: ProbabilisticDoubleInterval = null,
-                                udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] =
+                                udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] = {
     flatMapJava(toSerializableFlatteningFunction(udf), selectivity, udfLoad)
+  }
 
   /**
     * Feed this instance into a [[FlatMapOperator]].
@@ -199,13 +180,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     */
   def flatMapJava[NewOut: ClassTag](udf: SerializableFunction[Out, JavaIterable[NewOut]],
                                     selectivity: ProbabilisticDoubleInterval = null,
-                                    udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut] = {
-    val flatMapOperator = new FlatMapOperator(new FlatMapDescriptor(
-      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad
-    ))
-    this.connectTo(flatMapOperator, 0)
-    flatMapOperator
-  }
+                                    udfLoad: LoadProfileEstimator = null): DataQuanta[NewOut]
 
   /**
     * Feed this instance into a [[SampleOperator]]. If this operation is inside of a loop, the sampling size
@@ -220,7 +195,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
              datasetSize: Long = SampleOperator.UNKNOWN_DATASET_SIZE,
              seed: Option[Long] = None,
              sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuanta[Out] =
-    this.sampleDynamic(_ => sampleSize, datasetSize, seed, sampleMethod)
+    sampleDynamic(_ => sampleSize, datasetSize, seed, sampleMethod)
 
   /**
     * Feed this instance into a [[SampleOperator]]. If this operation is inside of a loop, the sampling size
@@ -234,8 +209,8 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
   def sampleDynamic(sampleSizeFunction: Int => Int,
                     datasetSize: Long = SampleOperator.UNKNOWN_DATASET_SIZE,
                     seed: Option[Long] = None,
-                    sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuanta[Out] =
-    this.sampleDynamicJava(
+                    sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuanta[Out] = {
+    sampleDynamicJava(
       new IntUnaryOperator {
         override def applyAsInt(operand: Int): Int = sampleSizeFunction(operand)
       },
@@ -243,6 +218,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
       seed,
       sampleMethod
     )
+  }
 
 
   /**
@@ -256,29 +232,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
   def sampleDynamicJava(sampleSizeFunction: IntUnaryOperator,
                         datasetSize: Long = SampleOperator.UNKNOWN_DATASET_SIZE,
                         seed: Option[Long] = None,
-                        sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuanta[Out] = {
-    if (seed.isEmpty) {
-      val sampleOperator = new SampleOperator(
-        sampleSizeFunction,
-        dataSetType[Out],
-        sampleMethod
-      )
-      sampleOperator.setDatasetSize(datasetSize)
-      this.connectTo(sampleOperator, 0)
-      sampleOperator
-    }
-    else {
-      val sampleOperator = new SampleOperator(
-        sampleSizeFunction,
-        dataSetType[Out],
-        sampleMethod,
-        seed.get
-      )
-      sampleOperator.setDatasetSize(datasetSize)
-      this.connectTo(sampleOperator, 0)
-      sampleOperator
-    }
-  }
+                        sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuanta[Out]
 
   /**
     * Assigns this instance a key extractor, which enables some key-based operations.
@@ -287,7 +241,10 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param keyExtractor extracts the key from the [[DataQuanta]]
     * @return the [[KeyedDataQuanta]]
     */
-  def keyBy[Key: ClassTag](keyExtractor: Out => Key) = new KeyedDataQuanta[Out, Key](this, keyExtractor)
+  //TODO: may need to be build by the extenders
+  def keyBy[Key: ClassTag](keyExtractor: Out => Key) : KeyedDataQuanta[Out, Key] = {
+    keyByJava(keyExtractor)
+  }
 
   /**
     * Assigns this instance a key extractor, which enables some key-based operations.
@@ -296,7 +253,8 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param keyExtractor extracts the key from the [[DataQuanta]]
     * @return the [[KeyedDataQuanta]]
     */
-  def keyByJava[Key: ClassTag](keyExtractor: SerializableFunction[Out, Key]) = new KeyedDataQuanta[Out, Key](this, keyExtractor)
+  //TODO: may need to be build by the extenders
+  def keyByJava[Key: ClassTag](keyExtractor: SerializableFunction[Out, Key]) : KeyedDataQuanta[Out, Key]
 
   /**
     * Feed this instance into a [[ReduceByOperator]].
@@ -308,8 +266,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     */
   def reduceByKey[Key: ClassTag](keyUdf: Out => Key,
                                  udf: (Out, Out) => Out,
-                                 udfLoad: LoadProfileEstimator = null): DataQuanta[Out] =
+                                 udfLoad: LoadProfileEstimator = null): DataQuanta[Out] = {
     reduceByKeyJava(toSerializableFunction(keyUdf), toSerializableBinaryOperator(udf), udfLoad)
+  }
 
   /**
     * Feed this instance into a [[ReduceByOperator]].
@@ -321,15 +280,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     */
   def reduceByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
                                      udf: SerializableBinaryOperator[Out],
-                                     udfLoad: LoadProfileEstimator = null)
-  : DataQuanta[Out] = {
-    val reduceByOperator = new ReduceByOperator(
-      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
-      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
-    )
-    this.connectTo(reduceByOperator, 0)
-    reduceByOperator
-  }
+                                     udfLoad: LoadProfileEstimator = null) : DataQuanta[Out]
 
   /**
     * Feed this instance into a [[MaterializedGroupByOperator]].
@@ -350,15 +301,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @return a new instance representing the [[MaterializedGroupByOperator]]'s output
     */
   def groupByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
-                                    keyUdfLoad: LoadProfileEstimator = null): DataQuanta[java.lang.Iterable[Out]] = {
-    val groupByOperator = new MaterializedGroupByOperator(
-      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key], keyUdfLoad),
-      dataSetType[Out],
-      groupedDataSetType[Out]
-    )
-    this.connectTo(groupByOperator, 0)
-    groupByOperator
-  }
+                                    keyUdfLoad: LoadProfileEstimator = null): DataQuanta[java.lang.Iterable[Out]]
 
   /**
     * Feed this instance into a [[GlobalReduceOperator]].
@@ -379,24 +322,14 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @return a new instance representing the [[GlobalReduceOperator]]'s output
     */
   def reduceJava(udf: SerializableBinaryOperator[Out],
-                 udfLoad: LoadProfileEstimator = null): DataQuanta[Out] = {
-    val globalReduceOperator = new GlobalReduceOperator(
-      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
-    )
-    this.connectTo(globalReduceOperator, 0)
-    globalReduceOperator
-  }
+                 udfLoad: LoadProfileEstimator = null): DataQuanta[Out]
 
   /**
     * Feed this instance into a [[GlobalMaterializedGroupOperator]].
     *
     * @return a new instance representing the [[GlobalMaterializedGroupOperator]]'s output
     */
-  def group(): DataQuanta[java.lang.Iterable[Out]] = {
-    val groupOperator = new GlobalMaterializedGroupOperator(dataSetType[Out], groupedDataSetType[Out])
-    this.connectTo(groupOperator, 0)
-    groupOperator
-  }
+  def group(): DataQuanta[java.lang.Iterable[Out]]
 
   /**
     * Feed this instance and a further instance into a [[UnionAllOperator]].
@@ -404,13 +337,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param that the other instance to union with
     * @return a new instance representing the [[UnionAllOperator]]'s output
     */
-  def union(that: DataQuanta[Out]): DataQuanta[Out] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val unionAllOperator = new UnionAllOperator(dataSetType[Out])
-    this.connectTo(unionAllOperator, 0)
-    that.connectTo(unionAllOperator, 1)
-    unionAllOperator
-  }
+  def union(that: DataQuanta[Out]): DataQuanta[Out]
 
   /**
     * Feed this instance and a further instance into a [[IntersectOperator]].
@@ -418,13 +345,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param that the other instance to intersect with
     * @return a new instance representing the [[IntersectOperator]]'s output
     */
-  def intersect(that: DataQuanta[Out]): DataQuanta[Out] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val intersectOperator = new IntersectOperator(dataSetType[Out])
-    this.connectTo(intersectOperator, 0)
-    that.connectTo(intersectOperator, 1)
-    intersectOperator
-  }
+  def intersect(that: DataQuanta[Out]): DataQuanta[Out]
 
   /**
     * Feeds this and a further instance into a [[JoinOperator]].
@@ -434,12 +355,11 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
     * @return a new instance representing the [[JoinOperator]]'s output
     */
-  def join[ThatOut: ClassTag, Key: ClassTag]
-  (thisKeyUdf: Out => Key,
-   that: DataQuanta[ThatOut],
-   thatKeyUdf: ThatOut => Key)
-  : DataQuanta[WayangTuple2[Out, ThatOut]] =
+  def join[ThatOut: ClassTag, Key: ClassTag] (thisKeyUdf: Out => Key,
+                                              that: DataQuanta[ThatOut],
+                                              thatKeyUdf: ThatOut => Key): DataQuanta[WayangTuple2[Out, ThatOut]] = {
     joinJava(toSerializableFunction(thisKeyUdf), that, toSerializableFunction(thatKeyUdf))
+  }
 
   /**
     * Feeds this and a further instance into a [[JoinOperator]].
@@ -449,20 +369,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
     * @return a new instance representing the [[JoinOperator]]'s output
     */
-  def joinJava[ThatOut: ClassTag, Key: ClassTag]
-  (thisKeyUdf: SerializableFunction[Out, Key],
-   that: DataQuanta[ThatOut],
-   thatKeyUdf: SerializableFunction[ThatOut, Key])
-  : DataQuanta[WayangTuple2[Out, ThatOut]] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val joinOperator = new JoinOperator(
-      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
-      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
-    )
-    this.connectTo(joinOperator, 0)
-    that.connectTo(joinOperator, 1)
-    joinOperator
-  }
+  def joinJava[ThatOut: ClassTag, Key: ClassTag] (thisKeyUdf: SerializableFunction[Out, Key],
+                                                  that: DataQuanta[ThatOut],
+                                                  thatKeyUdf: SerializableFunction[ThatOut, Key]) : DataQuanta[WayangTuple2[Out, ThatOut]]
 
   /**
     * Feeds this and a further instance into a [[CoGroupOperator]].
@@ -472,12 +381,11 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
     * @return a new instance representing the [[CoGroupOperator]]'s output
     */
-  def coGroup[ThatOut: ClassTag, Key: ClassTag]
-  (thisKeyUdf: Out => Key,
-   that: DataQuanta[ThatOut],
-   thatKeyUdf: ThatOut => Key)
-  : DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] =
+  def coGroup[ThatOut: ClassTag, Key: ClassTag] (thisKeyUdf: Out => Key,
+                                                 that: DataQuanta[ThatOut],
+                                                  thatKeyUdf: ThatOut => Key): DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] = {
     coGroupJava(toSerializableFunction(thisKeyUdf), that, toSerializableFunction(thatKeyUdf))
+  }
 
   /**
     * Feeds this and a further instance into a [[CoGroupOperator]].
@@ -487,20 +395,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
     * @return a new instance representing the [[CoGroupOperator]]'s output
     */
-  def coGroupJava[ThatOut: ClassTag, Key: ClassTag]
-  (thisKeyUdf: SerializableFunction[Out, Key],
-   that: DataQuanta[ThatOut],
-   thatKeyUdf: SerializableFunction[ThatOut, Key])
-  : DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val coGroupOperator = new CoGroupOperator(
-      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
-      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
-    )
-    this.connectTo(coGroupOperator, 0)
-    that.connectTo(coGroupOperator, 1)
-    coGroupOperator
-  }
+  def coGroupJava[ThatOut: ClassTag, Key: ClassTag] (thisKeyUdf: SerializableFunction[Out, Key],
+                                                     that: DataQuanta[ThatOut],
+                                                     thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]]
 
   /**
     * Feeds this and a further instance into a [[SortOperator]].
@@ -508,10 +405,9 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param keyUdf UDF to extract key from data quanta in this instance
     * @return a new instance representing the [[SortOperator]]'s output
     */
-  def sort[Key: ClassTag]
-  (keyUdf: Out => Key)
-  : DataQuanta[Out] =
+  def sort[Key: ClassTag] (keyUdf: Out => Key): DataQuanta[Out] = {
     sortJava(toSerializableFunction(keyUdf))
+  }
 
   /**
     * Feeds this and a further instance into a [[SortOperator]].
@@ -519,16 +415,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param keyUdf UDF to extract key from data quanta in this instance
     * @return a new instance representing the [[SortOperator]]'s output
     */
-  def sortJava[Key: ClassTag]
-  (keyUdf: SerializableFunction[Out, Key])
-  : DataQuanta[Out] = {
-    val sortOperator = new SortOperator(new TransformationDescriptor(
-      keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]))
-    this.connectTo(sortOperator, 0)
-    sortOperator
-  }
-
-
+  def sortJava[Key: ClassTag] (keyUdf: SerializableFunction[Out, Key]) : DataQuanta[Out]
 
   /**
     * Feeds this and a further instance into a [[CartesianOperator]].
@@ -536,48 +423,28 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param that the other instance
     * @return a new instance representing the [[CartesianOperator]]'s output
     */
-  def cartesian[ThatOut: ClassTag](that: DataQuanta[ThatOut])
-  : DataQuanta[WayangTuple2[Out, ThatOut]] = {
-    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
-    val cartesianOperator = new CartesianOperator(dataSetType[Out], dataSetType[ThatOut])
-    this.connectTo(cartesianOperator, 0)
-    that.connectTo(cartesianOperator, 1)
-    cartesianOperator
-  }
+  def cartesian[ThatOut: ClassTag](that: DataQuanta[ThatOut]) : DataQuanta[WayangTuple2[Out, ThatOut]]
 
   /**
     * Feeds this instance into a [[ZipWithIdOperator]].
     *
     * @return a new instance representing the [[ZipWithIdOperator]]'s output
     */
-  def zipWithId: DataQuanta[WayangTuple2[java.lang.Long, Out]] = {
-    val zipWithIdOperator = new ZipWithIdOperator(dataSetType[Out])
-    this.connectTo(zipWithIdOperator, 0)
-    zipWithIdOperator
-  }
+  def zipWithId: DataQuanta[WayangTuple2[java.lang.Long, Out]]
 
   /**
     * Feeds this instance into a [[DistinctOperator]].
     *
     * @return a new instance representing the [[DistinctOperator]]'s output
     */
-  def distinct: DataQuanta[Out] = {
-    val distinctOperator = new DistinctOperator(dataSetType[Out])
-    this.connectTo(distinctOperator, 0)
-    distinctOperator
-  }
+  def distinct: DataQuanta[Out]
 
   /**
     * Feeds this instance into a [[CountOperator]].
     *
     * @return a new instance representing the [[CountOperator]]'s output
     */
-  def count: DataQuanta[java.lang.Long] = {
-    val countOperator = new CountOperator(dataSetType[Out])
-    this.connectTo(countOperator, 0)
-    countOperator
-  }
-
+  def count: DataQuanta[java.lang.Long]
 
   /**
     * Feeds this instance into a do-while loop (guarded by a [[DoWhileOperator]].
@@ -625,13 +492,13 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     this.connectTo(doWhileOperator, DoWhileOperator.INITIAL_INPUT_INDEX)
 
     // Create and wire the loop body.
-    val loopDataQuanta = new DataQuanta[Out](doWhileOperator, DoWhileOperator.ITERATION_OUTPUT_INDEX)
+    val loopDataQuanta = DataQuantaFactory.build[Out](doWhileOperator, DoWhileOperator.ITERATION_OUTPUT_INDEX)
     val iterationResults = bodyBuilder.apply(loopDataQuanta)
     iterationResults.getField0.connectTo(doWhileOperator, DoWhileOperator.ITERATION_INPUT_INDEX)
     iterationResults.getField1.connectTo(doWhileOperator, DoWhileOperator.CONVERGENCE_INPUT_INDEX)
 
     // Return the iteration result.
-    new DataQuanta[Out](doWhileOperator, DoWhileOperator.FINAL_OUTPUT_INDEX)
+    DataQuantaFactory.build[Out](doWhileOperator, DoWhileOperator.FINAL_OUTPUT_INDEX)
   }
 
   /**
@@ -661,12 +528,12 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     this.connectTo(repeatOperator, RepeatOperator.INITIAL_INPUT_INDEX)
 
     // Create and wire the loop body.
-    val loopDataQuanta = new DataQuanta[Out](repeatOperator, RepeatOperator.ITERATION_OUTPUT_INDEX)
+    val loopDataQuanta = DataQuantaFactory.build[Out](repeatOperator, RepeatOperator.ITERATION_OUTPUT_INDEX)
     val iterationResult = bodyBuilder.apply(loopDataQuanta)
     iterationResult.connectTo(repeatOperator, RepeatOperator.ITERATION_INPUT_INDEX)
 
     // Return the iteration result.
-    new DataQuanta[Out](repeatOperator, RepeatOperator.FINAL_OUTPUT_INDEX)
+    DataQuantaFactory.build[Out](repeatOperator, RepeatOperator.FINAL_OUTPUT_INDEX)
   }
 
   /**
@@ -730,6 +597,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     *
     * @param f the action to perform as Java 8 lambda expression
     */
+  //TODO validate if it is the correct way
   def foreachJava(f: Consumer[Out]): Unit = {
     val sink = new LocalCallbackSink(f, dataSetType[Out])
     sink.setName("foreach()")
@@ -744,6 +612,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     *
     * @return the data quanta
     */
+  //TODO validate if it is the correct way
   def collect(): Iterable[Out] = {
     // Set up the sink.
     val collector = new java.util.LinkedList[Out]()
@@ -780,6 +649,7 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
     * @param formatterUdf UDF to format data quanta to [[String]]s
     * @param udfLoad      optional [[LoadProfileEstimator]] for the `udf`
     */
+  //TODO validate if it is the correct way
   def writeTextFileJava(url: String,
                         formatterUdf: SerializableFunction[Out, String],
                         udfLoad: LoadProfileEstimator = null): Unit = {
@@ -869,7 +739,8 @@ class DataQuanta[Out: ClassTag](val operator: ElementaryOperator, outputIndex: I
 
 object DataQuanta {
 
-  def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuanta[_] =
-    new DataQuanta(output.getOwner.asInstanceOf[ElementaryOperator], output.getIndex)(ClassTag(output.getType.getDataUnitType.getTypeClass), planBuilder)
+  def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuanta[_] = {
+    DataQuantaFactory.build[T](output.getOwner.asInstanceOf[ElementaryOperator], output.getIndex)(ClassTag(output.getType.getDataUnitType.getTypeClass), planBuilder)
+  }
 
 }
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala
new file mode 100644
index 0000000..1f79ae7
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaDefault.scala
@@ -0,0 +1,409 @@
+/*
+ *   Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing,
+ *   software distributed under the License is distributed on an
+ *   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *   KIND, either express or implied.  See the License for the
+ *   specific language governing permissions and limitations
+ *   under the License.
+ */
+
+package org.apache.wayang.api.dataquanta
+
+import org.apache.wayang.api.{PlanBuilder, basicDataUnitType, dataSetType, groupedDataSetType, groupedDataUnitType, toConsumer, toSerializableBinaryOperator, toSerializableFlatteningFunction, toSerializableFunction, toSerializablePartitionFunction, toSerializablePredicate}
+import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
+import org.apache.wayang.basic.function.ProjectionDescriptor
+import org.apache.wayang.basic.operators._
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction, SerializablePredicate}
+import org.apache.wayang.core.function._
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
+import org.apache.wayang.core.plan.wayangplan._
+
+import java.lang
+import java.lang.{Iterable => JavaIterable}
+import java.util.function.{Consumer, IntUnaryOperator, Function => JavaFunction}
+import scala.reflect._
+
+/**
+ * Represents an intermediate result/data flow edge in a [[WayangPlan]].
+ *
+ * @param operator    a unary [[Operator]] that produces this instance
+ * @param ev$1        the data type of the elements in this instance
+ * @param planBuilder keeps track of the [[WayangPlan]] being build
+ */
+class DataQuantaDefault[Out: ClassTag]
+          (override val operator: ElementaryOperator, outputIndex: Int = 0)
+          (implicit override val  planBuilder: PlanBuilder)
+    extends DataQuanta[Out](operator, outputIndex) {
+
+  /**
+   * Feed this instance into a [[MapOperator]].
+   *
+   * @param udf     a Java 8 lambda expression as UDF for the [[MapOperator]]
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[MapOperator]]'s output
+   */
+  override def mapJava[NewOut: ClassTag](udf: SerializableFunction[Out, NewOut],
+                                         udfLoad: LoadProfileEstimator = null): DataQuantaDefault[NewOut] = {
+    val mapOperator = new MapOperator(new TransformationDescriptor(
+      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], udfLoad
+    ))
+    this.connectTo(mapOperator, 0)
+    DataQuantaDefault.wrap[NewOut](mapOperator)
+  }
+
+  /**
+   * Feed this instance into a [[MapPartitionsOperator]].
+   *
+   * @param udf         a Java 8 lambda expression as UDF for the [[MapPartitionsOperator]]
+   * @param selectivity selectivity of the UDF
+   * @param udfLoad     optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[MapOperator]]'s output
+   */
+  override def mapPartitionsJava[NewOut: ClassTag](udf: SerializableFunction[JavaIterable[Out], JavaIterable[NewOut]],
+                                                   selectivity: ProbabilisticDoubleInterval = null,
+                                                   udfLoad: LoadProfileEstimator = null): DataQuantaDefault[NewOut] = {
+    val mapOperator = new MapPartitionsOperator(
+      new MapPartitionsDescriptor(udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad)
+    )
+    this.connectTo(mapOperator, 0)
+    DataQuantaDefault.wrap[NewOut](mapOperator)
+  }
+
+  /**
+   * Feed this instance into a [[MapOperator]] with a [[ProjectionDescriptor]].
+   *
+   * @param fieldNames names of the fields to be projected
+   * @return a new instance representing the [[MapOperator]]'s output
+   */
+  override def project[NewOut: ClassTag](fieldNames: Seq[String]): DataQuantaDefault[NewOut] = {
+    val projectionOperator = new MapOperator(
+      new ProjectionDescriptor(basicDataUnitType[Out], basicDataUnitType[NewOut], fieldNames: _*)
+    )
+    this.connectTo(projectionOperator, 0)
+    DataQuantaDefault.wrap[NewOut](projectionOperator)
+  }
+
+  /**
+   * Feed this instance into a [[FilterOperator]].
+   *
+   * @param udf         UDF for the [[FilterOperator]]
+   * @param sqlUdf      UDF as SQL `WHERE` clause
+   * @param selectivity selectivity of the UDF
+   * @param udfLoad     optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[FilterOperator]]'s output
+   */
+  override def filterJava(udf: SerializablePredicate[Out],
+                          sqlUdf: String = null,
+                          selectivity: ProbabilisticDoubleInterval = null,
+                          udfLoad: LoadProfileEstimator = null): DataQuantaDefault[Out] = {
+    val filterOperator = new FilterOperator(new PredicateDescriptor(
+      udf, this.output.getType.getDataUnitType.toBasicDataUnitType, selectivity, udfLoad
+    ).withSqlImplementation(sqlUdf))
+    this.connectTo(filterOperator, 0)
+    DataQuantaDefault.wrap[Out](filterOperator)
+  }
+
+  /**
+   * Feed this instance into a [[FlatMapOperator]].
+   *
+   * @param udf         a Java 8 lambda expression as UDF for the [[FlatMapOperator]]
+   * @param selectivity selectivity of the UDF
+   * @param udfLoad     optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[FlatMapOperator]]'s output
+   */
+  override def flatMapJava[NewOut: ClassTag](udf: SerializableFunction[Out, JavaIterable[NewOut]],
+                                             selectivity: ProbabilisticDoubleInterval = null,
+                                             udfLoad: LoadProfileEstimator = null): DataQuantaDefault[NewOut] = {
+    val flatMapOperator = new FlatMapOperator(new FlatMapDescriptor(
+      udf, basicDataUnitType[Out], basicDataUnitType[NewOut], selectivity, udfLoad
+    ))
+    this.connectTo(flatMapOperator, 0)
+    DataQuantaDefault.wrap[NewOut](flatMapOperator)
+  }
+
+
+  /**
+   * Feed this instance into a [[SampleOperator]].
+   *
+   * @param sampleSizeFunction absolute size of the sample as a function of the current iteration number
+   * @param datasetSize        optional size of the dataset to be sampled
+   * @param sampleMethod       the [[SampleOperator.Methods]] to use for sampling
+   * @return a new instance representing the [[FlatMapOperator]]'s output
+   */
+  override def sampleDynamicJava(sampleSizeFunction: IntUnaryOperator,
+                                 datasetSize: Long = SampleOperator.UNKNOWN_DATASET_SIZE,
+                                 seed: Option[Long] = None,
+                                 sampleMethod: SampleOperator.Methods = SampleOperator.Methods.ANY): DataQuantaDefault[Out] = {
+    if (seed.isEmpty) {
+      val sampleOperator = new SampleOperator(
+        sampleSizeFunction,
+        dataSetType[Out],
+        sampleMethod
+      )
+      sampleOperator.setDatasetSize(datasetSize)
+      this.connectTo(sampleOperator, 0)
+      DataQuantaDefault.wrap[Out](sampleOperator)
+    }
+    else {
+      val sampleOperator = new SampleOperator(
+        sampleSizeFunction,
+        dataSetType[Out],
+        sampleMethod,
+        seed.get
+      )
+      sampleOperator.setDatasetSize(datasetSize)
+      this.connectTo(sampleOperator, 0)
+      DataQuantaDefault.wrap[Out](sampleOperator)
+    }
+  }
+
+  /**
+   * Assigns this instance a key extractor, which enables some key-based operations.
+   *
+   * @see KeyedDataQuanta
+   * @param keyExtractor extracts the key from the [[DataQuantaDefault]]
+   * @return the [[KeyedDataQuanta]]
+   */
+  //TODO validate this implementation
+  override def keyByJava[Key: ClassTag](keyExtractor: SerializableFunction[Out, Key]) : KeyedDataQuanta[Out, Key] = {
+    new KeyedDataQuanta[Out, Key](this, keyExtractor)
+  }
+
+  /**
+   * Feed this instance into a [[ReduceByOperator]].
+   *
+   * @param keyUdf  UDF to extract the grouping key from the data quanta
+   * @param udf     aggregation UDF for the [[ReduceByOperator]]
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[ReduceByOperator]]'s output
+   */
+  override def reduceByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
+                                              udf: SerializableBinaryOperator[Out],
+                                              udfLoad: LoadProfileEstimator = null)
+  : DataQuantaDefault[Out] = {
+    val reduceByOperator = new ReduceByOperator(
+      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
+    )
+    this.connectTo(reduceByOperator, 0)
+    DataQuantaDefault.wrap[Out](reduceByOperator)
+  }
+
+  /**
+   * Feed this instance into a [[MaterializedGroupByOperator]].
+   *
+   * @param keyUdf     UDF to extract the grouping key from the data quanta
+   * @param keyUdfLoad optional [[LoadProfileEstimator]] for the `keyUdf`
+   * @return a new instance representing the [[MaterializedGroupByOperator]]'s output
+   */
+  override def groupByKeyJava[Key: ClassTag](keyUdf: SerializableFunction[Out, Key],
+                                             keyUdfLoad: LoadProfileEstimator = null): DataQuantaDefault[java.lang.Iterable[Out]] = {
+    val groupByOperator = new MaterializedGroupByOperator(
+      new TransformationDescriptor(keyUdf, basicDataUnitType[Out], basicDataUnitType[Key], keyUdfLoad),
+      dataSetType[Out],
+      groupedDataSetType[Out]
+    )
+    this.connectTo(groupByOperator, 0)
+    DataQuantaDefault.wrap[java.lang.Iterable[Out]](groupByOperator)
+  }
+
+  /**
+   * Feed this instance into a [[GlobalReduceOperator]].
+   *
+   * @param udf     aggregation UDF for the [[GlobalReduceOperator]]
+   * @param udfLoad optional [[LoadProfileEstimator]] for the `udf`
+   * @return a new instance representing the [[GlobalReduceOperator]]'s output
+   */
+  override def reduceJava(udf: SerializableBinaryOperator[Out],
+                          udfLoad: LoadProfileEstimator = null): DataQuantaDefault[Out] = {
+    val globalReduceOperator = new GlobalReduceOperator(
+      new ReduceDescriptor(udf, groupedDataUnitType[Out], basicDataUnitType[Out], udfLoad)
+    )
+    this.connectTo(globalReduceOperator, 0)
+    DataQuantaDefault.wrap[Out](globalReduceOperator)
+  }
+
+  /**
+   * Feeds this and a further instance into a [[JoinOperator]].
+   *
+   * @param thisKeyUdf UDF to extract keys from data quanta in this instance
+   * @param that       the other instance
+   * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
+   * @return a new instance representing the [[JoinOperator]]'s output
+   */
+  override def joinJava[ThatOut: ClassTag, Key: ClassTag](thisKeyUdf: SerializableFunction[Out, Key], that: DataQuanta[ThatOut], thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuanta[WayangTuple2[Out, ThatOut]] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val joinOperator = new JoinOperator(
+      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
+    )
+    this.connectTo(joinOperator, 0)
+    that.connectTo(joinOperator, 1)
+    DataQuantaDefault.wrap[WayangTuple2[Out, ThatOut]](joinOperator)
+  }
+
+  /**
+   * Feeds this and a further instance into a [[CoGroupOperator]].
+   *
+   * @param thisKeyUdf UDF to extract keys from data quanta in this instance
+   * @param that       the other instance
+   * @param thatKeyUdf UDF to extract keys from data quanta from `that` instance
+   * @return a new instance representing the [[CoGroupOperator]]'s output
+   */
+  override def coGroupJava[ThatOut: ClassTag, Key: ClassTag](thisKeyUdf: SerializableFunction[Out, Key], that: DataQuanta[ThatOut], thatKeyUdf: SerializableFunction[ThatOut, Key]): DataQuanta[WayangTuple2[JavaIterable[Out], JavaIterable[ThatOut]]] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val coGroupOperator = new CoGroupOperator(
+      new TransformationDescriptor(thisKeyUdf, basicDataUnitType[Out], basicDataUnitType[Key]),
+      new TransformationDescriptor(thatKeyUdf, basicDataUnitType[ThatOut], basicDataUnitType[Key])
+    )
+    this.connectTo(coGroupOperator, 0)
+    that.connectTo(coGroupOperator, 1)
+    DataQuantaDefault.wrap[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]](coGroupOperator)
+  }
+
+
+  /**
+   * Feeds this and a further instance into a [[SortOperator]].
+   *
+   * @param keyUdf UDF to extract key from data quanta in this instance
+   * @return a new instance representing the [[SortOperator]]'s output
+   */
+  override def sortJava[Key: ClassTag]
+  (keyUdf: SerializableFunction[Out, Key])
+  : DataQuantaDefault[Out] = {
+    val sortOperator = new SortOperator(new TransformationDescriptor(
+      keyUdf, basicDataUnitType[Out], basicDataUnitType[Key]))
+    this.connectTo(sortOperator, 0)
+    DataQuantaDefault.wrap[Out](sortOperator)
+  }
+
+
+
+  /**
+   * Broadcasts the data quanta in this instance to a further instance.
+   *
+   * @param receiver      the instance that receives the broadcast
+   * @param broadcastName the name with that the broadcast will be registered
+   */
+  private def broadcast(receiver: DataQuantaDefault[_], broadcastName: String) =
+    receiver.registerBroadcast(this.operator, this.outputIndex, broadcastName)
+
+  /**
+   * Register a further instance as broadcast.
+   *
+   * @param sender        provides the broadcast data quanta
+   * @param outputIndex   identifies the output index of the sender
+   * @param broadcastName the name with that the broadcast will be registered
+   */
+  private def registerBroadcast(sender: Operator, outputIndex: Int, broadcastName: String) =
+    sender.broadcastTo(outputIndex, this.operator, broadcastName)
+
+  /**
+   * Feed this instance into a [[GlobalMaterializedGroupOperator]].
+   *
+   * @return a new instance representing the [[GlobalMaterializedGroupOperator]]'s output
+   */
+  override def group(): DataQuanta[JavaIterable[Out]] = {
+    val groupOperator = new GlobalMaterializedGroupOperator(dataSetType[Out], groupedDataSetType[Out])
+    this.connectTo(groupOperator, 0)
+    DataQuantaDefault.wrap[JavaIterable[Out]](groupOperator)
+  }
+
+  /**
+   * Feed this instance and a further instance into a [[UnionAllOperator]].
+   *
+   * @param that the other instance to union with
+   * @return a new instance representing the [[UnionAllOperator]]'s output
+   */
+  override def union(that: DataQuanta[Out]): DataQuanta[Out] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val unionAllOperator = new UnionAllOperator(dataSetType[Out])
+    this.connectTo(unionAllOperator, 0)
+    that.connectTo(unionAllOperator, 1)
+    DataQuantaDefault.wrap[Out](unionAllOperator)
+  }
+
+  /**
+   * Feed this instance and a further instance into a [[IntersectOperator]].
+   *
+   * @param that the other instance to intersect with
+   * @return a new instance representing the [[IntersectOperator]]'s output
+   */
+  override def intersect(that: DataQuanta[Out]): DataQuanta[Out] =  {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val intersectOperator = new IntersectOperator(dataSetType[Out])
+    this.connectTo(intersectOperator, 0)
+    that.connectTo(intersectOperator, 1)
+    DataQuantaDefault.wrap[Out](intersectOperator)
+  }
+
+  /**
+   * Feeds this and a further instance into a [[CartesianOperator]].
+   *
+   * @param that the other instance
+   * @return a new instance representing the [[CartesianOperator]]'s output
+   */
+  override def cartesian[ThatOut: ClassTag](that: DataQuanta[ThatOut]): DataQuanta[WayangTuple2[Out, ThatOut]] = {
+    require(this.planBuilder eq that.planBuilder, s"$this and $that must use the same plan builders.")
+    val cartesianOperator = new CartesianOperator(dataSetType[Out], dataSetType[ThatOut])
+    this.connectTo(cartesianOperator, 0)
+    that.connectTo(cartesianOperator, 1)
+    DataQuantaDefault.wrap[WayangTuple2[Out, ThatOut]](cartesianOperator)
+  }
+
+  /**
+   * Feeds this instance into a [[ZipWithIdOperator]].
+   *
+   * @return a new instance representing the [[ZipWithIdOperator]]'s output
+   */
+  override def zipWithId: DataQuanta[WayangTuple2[lang.Long, Out]] = {
+    val zipWithIdOperator = new ZipWithIdOperator(dataSetType[Out])
+    this.connectTo(zipWithIdOperator, 0)
+    DataQuantaDefault.wrap[WayangTuple2[lang.Long, Out]](zipWithIdOperator)
+  }
+
+  /**
+   * Feeds this instance into a [[DistinctOperator]].
+   *
+   * @return a new instance representing the [[DistinctOperator]]'s output
+   */
+  override def distinct: DataQuanta[Out] = {
+    val distinctOperator = new DistinctOperator(dataSetType[Out])
+    this.connectTo(distinctOperator, 0)
+    DataQuantaDefault.wrap[Out](distinctOperator)
+  }
+
+  /**
+   * Feeds this instance into a [[CountOperator]].
+   *
+   * @return a new instance representing the [[CountOperator]]'s output
+   */
+  override def count: DataQuanta[lang.Long] = {
+    val countOperator = new CountOperator(dataSetType[Out])
+    this.connectTo(countOperator, 0)
+    DataQuantaDefault.wrap[lang.Long](countOperator)
+  }
+}
+
+object DataQuantaDefault {
+
+  def wrap[T:ClassTag](operator: ElementaryOperator, outputIndex: Int = 0)(implicit planBuilder: PlanBuilder): DataQuantaDefault[T] = {
+    new DataQuantaDefault[T](operator, outputIndex)
+  }
+
+  def create[T](output: OutputSlot[T])(implicit planBuilder: PlanBuilder): DataQuantaDefault[_] =
+    new DataQuantaDefault(output.getOwner.asInstanceOf[ElementaryOperator], output.getIndex)(ClassTag(output.getType.getDataUnitType.getTypeClass), planBuilder)
+
+}
+
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala
new file mode 100644
index 0000000..aad0e42
--- /dev/null
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/DataQuantaFactory.scala
@@ -0,0 +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.wayang.api.dataquanta
+
+import org.apache.wayang.api.PlanBuilder
+import org.apache.wayang.core.plan.wayangplan.ElementaryOperator
+
+import scala.reflect.ClassTag
+
+/**
+ * Because the [[DataQuanta]] could be implemented by several instances, then is need to have a factory
+ * that know what is the instance that need to build. To know which is the kind that need to be produced
+ * the [[DataQuantaFactory]] read a configuration files and create the right instance
+ */
+object DataQuantaFactory {
+
+  /**
+   * Given the configuration loaded the [[DataQuantaFactory.build()]] the right extender, if not configuration is
+   * provided the [[DataQuantaFactory]] will create a [[DataQuantaDefault]] instance
+   *
+   * @param operator is the operator that will be wrapped
+   * @param outputIndex index of the operator that will be used
+   * @param planBuilder implicit [[PlanBuilder]]
+   * @tparam T type that is process by the operator
+   * @return Instance of [[DataQuanta]] depending on the configurations provided
+   */
+  def build[T:ClassTag](operator: ElementaryOperator, outputIndex: Int = 0)(implicit planBuilder: PlanBuilder): DataQuanta[T] = {
+    //TODO validate if the correct way
+    DataQuantaDefault.wrap[T](operator, outputIndex)
+  }
+
+}
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.scala
index 99768cc..19410cf 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/JoinedDataQuanta.scala
@@ -21,7 +21,7 @@ package org.apache.wayang.api.dataquanta
 
 import org.apache.wayang.core.optimizer.costs.LoadProfileEstimator
 import org.apache.wayang.basic.data.{Tuple2 => WayangTuple2}
-
+import java.util.function.{ BiFunction => JavaBiFunction }
 import scala.reflect.ClassTag
 
 /**
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.scala
index fb6e3f1..71d909b 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/dataquanta/KeyedDataQuanta.scala
@@ -36,9 +36,9 @@ class KeyedDataQuanta[Out: ClassTag, Key: ClassTag](val dataQuanta: DataQuanta[O
    * @param that the other [[KeyedDataQuanta]] to join with
    * @return the join product [[DataQuanta]]
    */
-  def join[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]):
-  DataQuanta[WayangTuple2[Out, ThatOut]] =
+  def join[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]): DataQuanta[WayangTuple2[Out, ThatOut]] = {
     dataQuanta.joinJava[ThatOut, Key](this.keyExtractor, that.dataQuanta, that.keyExtractor)
+  }
 
   /**
    * Performs a co-group. The grouping fields are governed by the [[KeyedDataQuanta]]'s keys.
@@ -47,7 +47,8 @@ class KeyedDataQuanta[Out: ClassTag, Key: ClassTag](val dataQuanta: DataQuanta[O
    * @return the co-grouped [[DataQuanta]]
    */
   def coGroup[ThatOut: ClassTag](that: KeyedDataQuanta[ThatOut, Key]):
-  DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] =
+  DataQuanta[WayangTuple2[java.lang.Iterable[Out], java.lang.Iterable[ThatOut]]] = {
     dataQuanta.coGroupJava[ThatOut, Key](this.keyExtractor, that.dataQuanta, that.keyExtractor)
+  }
 
 }
diff --git a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala
index 7dfefd0..0de9a4a 100644
--- a/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala
+++ b/wayang-api/wayang-api-scala-java/src/main/scala/org/apache/wayang/api/package.scala
@@ -18,7 +18,7 @@
 
 package org.apache.wayang
 
-import org.apache.wayang.api.dataquanta.{DataQuanta, JoinedDataQuanta}
+import org.apache.wayang.api.dataquanta.{DataQuanta, DataQuantaFactory, JoinedDataQuanta}
 
 import _root_.java.lang.{Class => JavaClass, Iterable => JavaIterable}
 import _root_.java.util.function.{Consumer, ToLongBiFunction, ToLongFunction}
@@ -142,10 +142,12 @@ package object api {
 
   implicit def createPlanBuilder(wayangContext: WayangContext): PlanBuilder = new PlanBuilder(wayangContext)
 
-  implicit private[api] def wrap[Out: ClassTag](op: ElementaryOperator)(implicit planBuilder: PlanBuilder): DataQuanta[Out] =
-    new DataQuanta[Out](op)
+  implicit private[api] def wrap[Out: ClassTag](op: ElementaryOperator)(implicit planBuilder: PlanBuilder): DataQuanta[Out] = {
+    DataQuantaFactory.build[Out](op)
+  }
 
-  implicit def elevateRecordDataQuanta(dataQuanta: DataQuanta[Record]): RecordDataQuanta =
+  implicit def elevateRecordDataQuanta(dataQuanta: DataQuanta[Record]): RecordDataQuanta = {
     new RecordDataQuanta(dataQuanta)
+  }
 
 }

[incubator-wayang] 15/15: Merge pull request #29 from apache/debugger-sidecar

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 6cfa2d88d53a822fc5fb0fba046780b855e24022
Merge: a272d73 e3c727c
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Wed Sep 22 17:20:05 2021 +0200

    Merge pull request #29 from apache/debugger-sidecar
    
    Debugger sidecar its merged because it just to clean branch that are for the same implementations

 .../wayang-hackit/wayang-hackit-sidecar/pom.xml    |  55 ++++-
 .../wayang/hackit/sidecar/HackitSidecar.java       |  12 ++
 .../org/apache/wayang/hackit/sidecar/test.java     | 225 +++++++++++++++++++++
 .../hackit/sidecar/webservice/ExecutorManager.java | 107 ++++++++++
 .../hackit/sidecar/webservice/WebService.java      | 224 ++++++++++++++++++++
 .../src/main/resources/application.properties      |   1 +
 .../src/main/resources/claim.yaml                  |  11 +
 .../src/main/resources/jobmanager-service.yaml     |  18 ++
 .../src/main/resources/jobmanager.yaml             |  33 +++
 .../src/main/resources/jobmanager_new.yaml         |  40 ++++
 .../src/main/resources/loadbalancer.yaml           |  14 ++
 .../src/main/resources/taskmanager.yaml            |  31 +++
 .../src/main/resources/taskmanager_new.yaml        |  38 ++++
 .../src/main/resources/volume.yaml                 |  14 ++
 14 files changed, 820 insertions(+), 3 deletions(-)

[incubator-wayang] 01/15: [WAYANG-28] Add javadoc to tags package

Posted by be...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bertty pushed a commit to branch WAYANG-28
in repository https://gitbox.apache.org/repos/asf/incubator-wayang.git

commit 12d9afa9e406195ae030ae7d0a009e1ade919bb7
Author: Bertty Contreras-Rojas <be...@scalytics.io>
AuthorDate: Tue May 4 12:14:05 2021 -0400

    [WAYANG-28] Add javadoc to tags package
---
 .../wayang/plugin/hackit/core/tags/CrashTag.java   | 15 ++++++++++
 .../wayang/plugin/hackit/core/tags/DebugTag.java   | 15 ++++++++++
 .../wayang/plugin/hackit/core/tags/DisplayTag.java | 16 ++++++++++
 .../wayang/plugin/hackit/core/tags/HackitTag.java  | 34 ++++++++++++++++++++++
 .../plugin/hackit/core/tags/HackitTagBuilder.java  |  1 +
 .../wayang/plugin/hackit/core/tags/LogTag.java     | 17 +++++++++++
 .../wayang/plugin/hackit/core/tags/PauseTag.java   | 16 ++++++++++
 .../wayang/plugin/hackit/core/tags/SkipTag.java    | 17 +++++++++++
 .../wayang/plugin/hackit/core/tags/TraceTag.java   | 16 ++++++++++
 9 files changed, 147 insertions(+)

diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/CrashTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/CrashTag.java
index 94fd98e..dca602a 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/CrashTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/CrashTag.java
@@ -17,31 +17,46 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * CrashTag is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple} that
+ * produce and exception that goes from UDF context and could make that the process stop.
+ */
 public class CrashTag extends HackitTag {
 
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static CrashTag seed = null;
 
+    /**
+     * default construct
+     */
     private CrashTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DebugTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DebugTag.java
index ec2ad70..7c7a834 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DebugTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DebugTag.java
@@ -17,31 +17,46 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * DebugTag is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * as an element that need to capture the timestamp and send it out to analyse the values
+ */
 public class DebugTag extends HackitTag  {
 
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static DebugTag seed = null;
 
+    /**
+     * default construct
+     */
     private DebugTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DisplayTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DisplayTag.java
index 19a673d..d584159 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DisplayTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/DisplayTag.java
@@ -17,30 +17,46 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * Display is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * as an element that need to be sent it out to just print the values
+ */
 public class DisplayTag extends HackitTag  {
+
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static DisplayTag seed = null;
 
+    /**
+     * default construct
+     */
     private DisplayTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTag.java
index 6fdcdb4..9d4c813 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTag.java
@@ -26,16 +26,38 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+/**
+ * HackitTag extend from {@link ActionGroup} becuase is the element that allows to create
+ * an element that contains the flags for the {@link org.apache.wayang.plugin.hackit.core.tagger.HackitTagger}
+ * and {@link org.apache.wayang.plugin.hackit.core.sniffer.HackitSniffer} could perform the right logic
+ * to {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple} that have the tag
+ *
+ * HachitTag implements {@link Serializable} because the element could be send it out.
+ */
 public abstract class HackitTag implements Serializable, ActionGroup {
 
+    /**
+     * TODO: add comment, validate if apply for the use case
+     */
     private List<TaggerFunction<?>> callbacks;
+
+    /**
+     * TODO: add comment, validate if apply for the use case
+     */
     private Map<String, Object> callback_results;
 
+    /**
+     * Default Construct
+     */
     protected HackitTag(){
+        //TODO: validate is is not better to go by delay option
         this.callbacks = new ArrayList<>();
         this.callback_results = new HashMap<>();
     }
 
+    /**
+     * TODO: add comment, validate if apply for the use case
+     */
     public void callback(){
         if(hasCallback()) {
             this.callbacks.stream().forEach(
@@ -43,10 +65,22 @@ public abstract class HackitTag implements Serializable, ActionGroup {
             );
         }
     }
+
+    /**
+     * TODO: add comment, validate if apply for the use case
+     *
+     * @param name
+     * @param value
+     */
     public void addValue(String name, Object value){
         this.callback_results.put(name, value);
     }
 
+    /**
+     * TODO: add comment, validate if apply for the use case
+     *
+     * @return
+     */
     public abstract HackitTag getInstance();
 
     @Override
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTagBuilder.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTagBuilder.java
index e80a33d..3daffa0 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTagBuilder.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/HackitTagBuilder.java
@@ -17,6 +17,7 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+//TODO: all the tags need to be generated by a builder
 public class HackitTagBuilder {
     //TODO check how to implement wildfly
 }
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/LogTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/LogTag.java
index fa1020b..d00805e 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/LogTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/LogTag.java
@@ -17,30 +17,47 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * LogTag is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * as part of logging status, this will produce to send out just the metadata
+ */
 public class LogTag extends HackitTag  {
+
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static LogTag seed = null;
 
+    /**
+     * default construct
+     */
     public LogTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/PauseTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/PauseTag.java
index 21daab8..31e9515 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/PauseTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/PauseTag.java
@@ -17,30 +17,46 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * PauseTag is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * where is need to pause all the process and wait for a signal that tell what to do, either kill the process or continues
+ */
 public class PauseTag extends HackitTag  {
+
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static PauseTag seed = null;
 
+    /**
+     * default construct
+     */
     private PauseTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/SkipTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/SkipTag.java
index 1e9dd1d..b8523f9 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/SkipTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/SkipTag.java
@@ -17,30 +17,47 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * SkipTag is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * that not need to be process by the {@link org.apache.wayang.plugin.hackit.core.sniffer.HackitSniffer} and
+ * also need to be removed and not continues with the process.
+ */
 public class SkipTag extends HackitTag  {
+
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static SkipTag seed = null;
 
+    /**
+     * default construct
+     */
     private SkipTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }
 
diff --git a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/TraceTag.java b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/TraceTag.java
index 1482f52..47c1108 100644
--- a/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/TraceTag.java
+++ b/wayang-plugins/wayang-hackit/wayang-hackit-core/src/main/java/org/apache/wayang/plugin/hackit/core/tags/TraceTag.java
@@ -17,30 +17,46 @@
  */
 package org.apache.wayang.plugin.hackit.core.tags;
 
+/**
+ * DebugTag is the {@link HackitTag} that identify the {@link org.apache.wayang.plugin.hackit.core.tuple.HackitTuple}
+ * as an element which will be identifier and also add the timestamp to be enable to follow it
+ */
 public class TraceTag extends HackitTag  {
+
+    /**
+     * seed is the element that allow to have small overhead in memory, is like use
+     * singleton.
+     */
     private static TraceTag seed = null;
 
+    /**
+     * default construct
+     */
     private TraceTag(){
         super();
     }
 
     @Override
     public boolean isSendOut() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isSkip() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean isHaltJob() {
+        //TODO: set the correct value to this return
         return false;
     }
 
     @Override
     public boolean hasCallback() {
+        //TODO: set the correct value to this return
         return false;
     }