beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-5393) Euphoria: Full windowing support
Date Fri, 21 Sep 2018 10:28:02 GMT

     [ https://issues.apache.org/jira/browse/BEAM-5393?focusedWorklogId=146321&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-146321 ]

ASF GitHub Bot logged work on BEAM-5393:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 21/Sep/18 10:27
            Start Date: 21/Sep/18 10:27
    Worklog Time Spent: 10m 
      Work Description: dmvk closed pull request #6462: [BEAM-5393] Window builder
URL: https://github.com/apache/beam/pull/6462
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle
index b7c3659c196..48f684197e5 100644
--- a/sdks/java/core/build.gradle
+++ b/sdks/java/core/build.gradle
@@ -74,3 +74,18 @@ dependencies {
   shadowTest library.java.mockito_core
   shadowTest "com.esotericsoftware.kryo:kryo:2.21"
 }
+
+configurations {
+  testArtifact
+}
+
+task testJar(type: Jar) {
+  from sourceSets.test.output
+  classifier = 'test'
+}
+
+artifacts {
+  testArtifact testJar
+}
+
+
diff --git a/sdks/java/extensions/euphoria/README.md b/sdks/java/extensions/euphoria/README.md
new file mode 100644
index 00000000000..a1024019863
--- /dev/null
+++ b/sdks/java/extensions/euphoria/README.md
@@ -0,0 +1,159 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+-->
+
+
+# Euphoria Java 8 DSL
+
+Easy to use Java 8 DSL for the Beam Java SDK. Provides a high-level abstraction of Beam transformations, which is both easy to read and write. Can be used as a complement to existing Beam pipelines (convertible back and forth).
+
+Integration of Euphoria API to Beam is in **progress** ([BEAM-3900](https://issues.apache.org/jira/browse/BEAM-3900)).
+
+## How to build
+
+Euphoria is located in `dsl-euphoria` branch. To build `euphoria` subprojects use command:
+
+```
+./gradlew :beam-sdks-java-extensions-euphoria-beam:build 
+```
+
+## WordCount example
+
+```java
+Pipeline pipeline = Pipeline.create(options);
+
+// Transform to euphoria's flow.
+BeamFlow flow = BeamFlow.create(pipeline);
+
+// Source of data loaded from Beam IO.
+PCollection<String> input =
+    pipeline.apply(Create.of(dataSets)).setTypeDescriptor(TypeDescriptor.of(String.class));
+// Transform PCollection to euphoria's Dataset.
+Dataset<String> lines = flow.wrapped(input);
+
+// FlatMap processes one input element at a time and allows user code to emit
+// zero, one, or more output elements. From input lines we will get data set of words.
+Dataset<String> words = FlatMap.named("TOKENIZER")
+    .of(lines)
+    .using((String line, Collector<String> context) -> {
+      for (String word : line.split("\\s+")) {
+        context.collect(word);
+      }
+    })
+    .output();
+
+// From each input element we extract a key (word) and value, which is the constant `1`.
+// Then, we reduce by the key - the operator ensures that all values for the same
+// key end up being processed together. It applies user defined function (summing word counts for each
+// unique word) and its emitted to output. 
+Dataset<KV<String, Long>> counted = ReduceByKey.named("COUNT")
+    .of(words)
+    .keyBy(w -> w)
+    .valueBy(w -> 1L)
+    .combineBy(Sums.ofLongs())
+    .output();
+
+// Format output.
+Dataset<String> output = MapElements.named("FORMAT")
+    .of(counted)
+    .using(p -> p.getKey() + ": " + p.getValue())
+    .output();
+
+// Transform Dataset back to PCollection. It can be done in any step of this flow.
+PCollection<String> outputCollection = flow.unwrapped(output);
+
+// Now we can again use Beam transformation. In this case we save words and their count
+// into the text file.
+outputCollection.apply(TextIO.write().to(options.getOutput()));
+
+pipeline.run();
+```
+
+
+
+## Key features
+
+ * Unified API that supports both batch and stream processing using
+   the same code
+ * Avoids vendor lock-in - migrating between different engines is
+   matter of configuration
+ * Declarative Java API using Java 8 Lambda expressions
+ * Support for different notions of time (_event time, ingestion
+   time_)
+ * Flexible windowing (_Time, TimeSliding, Session, Count_)
+
+
+## Supported Engines
+
+Euphoria can be translated to Beam so it support same runners as Beam ([more info](https://beam.apache.org/documentation/runners/capability-matrix/))
+
+
+## Bugs / Features / Contributing
+
+There's still a lot of room for improvements and extensions.  Have a
+look into the [issue tracker](https://issues.apache.org/jira/browse/BEAM-3900)
+and feel free to contribute by reporting new problems, contributing to
+existing ones, or even open issues in case of questions.  Any constructive
+feedback is warmly welcome!
+
+As usually with open source, don't hesitate to fork the repo and
+submit a pull requests if you see something to be changed.  We'll be
+happy see euphoria improving over time.
+
+
+## Documentation
+_In progress_
+
+### Available transformations
+
+- `CountByKey`: Counting elements with same key.
+- `Distinct`: Outputting distinct (based on equals method) elements.
+- `Join` : Inner join of two datasets by given key producing single new dataset.
+- `LeftJoin` : Left outer join of two input datasets producing single new dataset.
+- `RightJoin`: Right outer join of two input datasets producing single new dataset.
+- `FullJoin`: Full outer join of two input datasets producing single new dataset.
+- `MapElements`: Simple one-to-one transformation of input elements.
+- `FlatMap` : A transformation of a dataset from one type into another allowing user code to generate zero,
+     one, or many output elements for a given input element.
+- `Filter` : Output elements that pass given condition.
+- `ReduceByKey`:Operator performing state-less aggregation by given reduce function. The reduction is performed
+                 on all extracted values on each key-window.
+- `ReduceWindow`: Reduces all elements in a window. 
+- `SumByKey`: summing of long values extracted from elements.
+- `TopPerKey` :  Emits top element for defined keys and windows.
+- `Union` : The union of at least two datasets of the same type.
+- `AssignEventTime`: A convenient alias for assignment of event time.
+
+
+Lower level transformations (if possible user should prefer above transformations):
+- `ReduceStateByKey`: assigns each input item to a set of windows and turns the item into a key/value pair.
+For each of the assigned windows the extracted value is accumulated using a user provided `StateFactory` state
+ implementation under the extracted key. I.e. the value is accumulated into a state identified by
+ a key/window pair.
+
+## Contact us
+
+* Feel free to open an issue in the [issue tracker](https://issues.apache.org/jira/browse/BEAM-3900)
+with filled `dsl-euphoria` component. 
+
+## License
+
+Euphoria is licensed under the terms of the Apache License 2.0.
+
+
+
diff --git a/sdks/java/extensions/euphoria/euphoria-core/build.gradle b/sdks/java/extensions/euphoria/euphoria-core/build.gradle
new file mode 100644
index 00000000000..36775886fb2
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/build.gradle
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+apply plugin: org.apache.beam.gradle.BeamModulePlugin
+applyJavaNature()
+
+description = "Apache Beam :: SDKs :: Java :: Extensions :: Euphoria Java 8 DSL :: Core"
+
+ext {
+    kryoVersion = '4.0.2'
+}
+
+dependencies {
+    shadow project(path: ":beam-sdks-java-core", configuration: "shadow")
+    shadow "com.esotericsoftware:kryo:${kryoVersion}"
+    shadow library.java.guava
+    testCompile library.java.mockito_core
+    testCompile project(path: ':beam-sdks-java-extensions-euphoria-testing')
+    testCompile library.java.slf4j_api
+    testCompile library.java.hamcrest_core
+    testCompile library.java.hamcrest_library
+    testCompile library.java.mockito_core
+    testCompile project(path: ":beam-sdks-java-core", configuration: "shadowTest")
+    testCompile project(':beam-runners-direct-java')
+}
+
+tasks.withType(JavaCompile) {
+    options.compilerArgs += ['-Xlint:deprecation']
+}
+
+test {
+    jvmArgs '-Dsun.io.serialization.extendedDebugInfo=true'
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/Audience.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/Audience.java
new file mode 100644
index 00000000000..02d90bd0ddc
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/Audience.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.annotation.audience;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/** Intended audience of API. */
+@Documented
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.SOURCE)
+public @interface Audience {
+
+  Type[] value();
+
+  /** Usage type of Audience. */
+  enum Type {
+    /** The API is intended to be used by client code. */
+    CLIENT,
+    /** The API is intended for use by executor code. */
+    EXECUTOR,
+    /** The API is intended for internal use. */
+    INTERNAL,
+    /** The API is intended for use primarily for tests. */
+    TESTS
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/package-info.java
new file mode 100644
index 00000000000..a1d9470ef24
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Intended audience of API. */
+package org.apache.beam.sdk.extensions.euphoria.core.annotation.audience;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Basic.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Basic.java
new file mode 100644
index 00000000000..600cc339b4d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Basic.java
@@ -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.beam.sdk.extensions.euphoria.core.annotation.operator;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation marking an operator a {@code Basic} operator. Such an operator *must* be implemented
+ * (natively) by an executor in order to run a flow.
+ */
+@Documented
+@Target(ElementType.TYPE)
+public @interface Basic {
+
+  /** @return the state complexity */
+  StateComplexity state();
+
+  /** @return number of global repartition operations */
+  int repartitions();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Derived.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Derived.java
new file mode 100644
index 00000000000..017ead756e6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Derived.java
@@ -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.beam.sdk.extensions.euphoria.core.annotation.operator;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Target;
+
+/**
+ * A {@code Derived} operator is operator that is efficiently implemented by the basic or
+ * recommended operators, so there is no explicit reason for the executor to implement it by hand.
+ */
+@Documented
+@Target(ElementType.TYPE)
+public @interface Derived {
+
+  /** @return the state complexity */
+  StateComplexity state();
+
+  /** @return the number of global repartition operations */
+  int repartitions();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Recommended.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Recommended.java
new file mode 100644
index 00000000000..1d738defcae
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Recommended.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Target;
+
+/**
+ * A {@code Recommended} operator is such an operator that is strongly advised to be implemented
+ * natively by executor due to performance reasons.
+ */
+@Documented
+@Target(ElementType.TYPE)
+public @interface Recommended {
+
+  /**
+   * @return a human readable explanation why the annotated operator is recommendation for native
+   *     implementation by an executor
+   */
+  String reason();
+
+  /** @return the state complexity */
+  StateComplexity state();
+
+  /** @return the number of global repartition operations */
+  int repartitions();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/StateComplexity.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/StateComplexity.java
new file mode 100644
index 00000000000..60506045b12
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/StateComplexity.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator;
+
+/** Space complexity of an operator's state in each window depending on the size of input. */
+public enum StateComplexity {
+
+  /** The size of state will be O(N) in the size of input. */
+  LINEAR,
+
+  /** The size of state will be sub-linear but not constant. */
+  SUBLINEAR,
+
+  /** The size of state will be O(1) in the size of input. */
+  CONSTANT,
+
+  /** There is no state in this operator. */
+  ZERO,
+
+  /**
+   * The size of state will be O(1) if the passed function is `combinable` (commutative,
+   * associative), otherwise it will be O(N).
+   */
+  CONSTANT_IF_COMBINABLE,
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java
new file mode 100644
index 00000000000..aede565279f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * This package contains a set of annotations wch allows to mark operators to explicitly tell that
+ * they have varions properties.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/Experimental.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/Experimental.java
new file mode 100644
index 00000000000..ec4d3e17e43
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/Experimental.java
@@ -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.beam.sdk.extensions.euphoria.core.annotation.stability;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Target;
+
+/**
+ * Any (API) item annotated with this annotation signals that the item is purely experimental and
+ * may be change or completely removed without any notice at any time.
+ *
+ * <p>Clients should generally avoid usage of such items, except when experimenting :)
+ */
+@Documented
+@Target({
+  ElementType.TYPE,
+  ElementType.FIELD,
+  ElementType.METHOD,
+  ElementType.PARAMETER,
+  ElementType.CONSTRUCTOR,
+  ElementType.ANNOTATION_TYPE,
+  ElementType.PACKAGE
+})
+public @interface Experimental {
+
+  /** @return a human readable hint about the experiment. */
+  String value() default "";
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java
new file mode 100644
index 00000000000..41f53b59688
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** API stability related annotations. */
+package org.apache.beam.sdk.extensions.euphoria.core.annotation.stability;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java
new file mode 100644
index 00000000000..4a5fbb9c78b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Accumulators collect values from user functions. Accumulators allow user to calculate statistics
+ * during the flow execution.
+ *
+ * <p>Accumulators are inspired by the Hadoop/MapReduce counters.
+ */
+@Audience(Audience.Type.INTERNAL)
+public interface Accumulator {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java
new file mode 100644
index 00000000000..bb4e1cd4065
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.EuphoriaOptions;
+
+/**
+ * Provides access to an accumulator backend service. It is intended to be implemented by third
+ * party to support different type of services.
+ */
+@Audience(Audience.Type.EXECUTOR)
+public interface AccumulatorProvider {
+
+  static AccumulatorProvider.Factory of(Pipeline pipeline) {
+    return pipeline.getOptions().as(EuphoriaOptions.class).getAccumulatorProviderFactory();
+  }
+
+  /**
+   * Get an existing instance of a counter or create a new one.
+   *
+   * @param name Unique name of the counter.
+   * @return Instance of a counter. @Deprecated use {@link #getCounter(String, String)} instead
+   */
+  Counter getCounter(String name);
+
+  /**
+   * Get an existing instance of a counter or create a new one.
+   *
+   * @param namespace of counter (e.g. operator name)
+   * @param name of the counter
+   * @return Instance of a counter.
+   */
+  Counter getCounter(String namespace, String name);
+  /**
+   * Get an existing instance of a histogram or create a new one.
+   *
+   * @param name Unique name of the histogram.
+   * @return Instance of a histogram. @Deprecated use {@link #getHistogram(String, String)} instead
+   */
+  Histogram getHistogram(String name);
+
+  /**
+   * Get an existing instance of a histogram or create a new one.
+   *
+   * @param namespace of histogram (e.g. operator name)
+   * @param name of the counter
+   * @return Instance of a counter.
+   */
+  Histogram getHistogram(String namespace, String name);
+
+  /**
+   * Get an existing instance of a timer or create a new one.
+   *
+   * @param name Unique name of the timer.
+   * @return Instance of a timer. @Deprecated use {@link #getHistogram(String, String)} instead
+   */
+  Timer getTimer(String name);
+
+  /**
+   * Creates a new instance of {@link AccumulatorProvider} initialized by given settings.
+   *
+   * <p>It is required this factory is thread-safe.
+   */
+  @FunctionalInterface
+  interface Factory extends Serializable {
+
+    AccumulatorProvider create();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java
new file mode 100644
index 00000000000..b5e93ef5d3b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Counter is a type of accumulator making a sum from integral numbers. */
+@Audience(Audience.Type.CLIENT)
+public interface Counter extends Accumulator {
+
+  /**
+   * Increment counter by given value.
+   *
+   * @param value Value to be added to the counter.
+   */
+  void increment(long value);
+
+  /** Increment counter by one. */
+  void increment();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Histogram.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Histogram.java
new file mode 100644
index 00000000000..18c71ede8a7
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Histogram.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Histogram is a type of accumulator recording a distribution of different values. */
+@Audience(Audience.Type.CLIENT)
+public interface Histogram extends Accumulator {
+
+  /**
+   * Add specified value.
+   *
+   * @param value Value to be added.
+   */
+  void add(long value);
+
+  /**
+   * Add specified value multiple times.
+   *
+   * @param value Value to be added.
+   * @param times Number of occurrences to add.
+   */
+  void add(long value, long times);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java
new file mode 100644
index 00000000000..538b908fcea
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators;
+
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Timer provides convenience API very similar to {@link Histogram} but extended by time unit
+ * support.
+ */
+@Audience(Audience.Type.CLIENT)
+public interface Timer extends Accumulator {
+
+  /**
+   * Add specific duration.
+   *
+   * @param duration Duration to be added.
+   */
+  void add(Duration duration);
+
+  /**
+   * Add specific duration with given time unit.
+   *
+   * @param duration Duration to be added.
+   * @param unit Time unit.
+   */
+  default void add(long duration, TimeUnit unit) {
+    add(Duration.ofMillis(unit.toMillis(duration)));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java
new file mode 100644
index 00000000000..154d66986a0
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java
@@ -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.beam.sdk.extensions.euphoria.core.client.accumulators;
+
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Placeholder implementation of {@link AccumulatorProvider} that may be used in executors as a
+ * default.
+ */
+@Audience(Audience.Type.EXECUTOR)
+public class VoidAccumulatorProvider implements AccumulatorProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VoidAccumulatorProvider.class);
+
+  private VoidAccumulatorProvider() {}
+
+  public static Factory getFactory() {
+    return Factory.get();
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return VoidCounter.INSTANCE;
+  }
+
+  @Override
+  public Counter getCounter(String namespace, String name) {
+    return VoidCounter.INSTANCE;
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    return VoidHistogram.INSTANCE;
+  }
+
+  @Override
+  public Histogram getHistogram(String namespace, String name) {
+    return VoidHistogram.INSTANCE;
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    return VoidTimer.INSTANCE;
+  }
+
+  // ------------------------
+
+  /** AccumulatorProvider Factory. */
+  public static class Factory implements AccumulatorProvider.Factory {
+
+    private static final Factory INSTANCE = new Factory();
+
+    private static final AccumulatorProvider PROVIDER = new VoidAccumulatorProvider();
+
+    private static final AtomicBoolean isLogged = new AtomicBoolean();
+
+    private Factory() {}
+
+    public static Factory get() {
+      return INSTANCE;
+    }
+
+    @Override
+    public AccumulatorProvider create() {
+      if (isLogged.compareAndSet(false, true)) {
+        LOG.warn("Using accumulators with VoidAccumulatorProvider will have no effect");
+      }
+      return PROVIDER;
+    }
+  }
+
+  // ------------------------
+
+  /** Empty implementation of Counter doesnt count anything. */
+  private static class VoidCounter implements Counter {
+
+    private static final VoidCounter INSTANCE = new VoidCounter();
+
+    private VoidCounter() {}
+
+    @Override
+    public void increment(long value) {
+      // NOOP
+    }
+
+    @Override
+    public void increment() {
+      // NOOP
+    }
+  }
+
+  /** Empty implementation of Histogram doesnt count anything. */
+  public static class VoidHistogram implements Histogram {
+
+    private static final VoidHistogram INSTANCE = new VoidHistogram();
+
+    private VoidHistogram() {}
+
+    @Override
+    public void add(long value) {
+      // NOOP
+    }
+
+    @Override
+    public void add(long value, long times) {
+      // NOOP
+    }
+  }
+
+  /** Empty implementation of Timer doesnt measure anything. */
+  public static class VoidTimer implements Timer {
+
+    private static final VoidTimer INSTANCE = new VoidTimer();
+
+    private VoidTimer() {}
+
+    @Override
+    public void add(Duration duration) {
+      // NOOP
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java
new file mode 100644
index 00000000000..8b7c77945f2
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Accumulators package. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java
new file mode 100644
index 00000000000..0096311d3eb
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.dataset;
+
+import java.util.Map;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * A dataset abstraction.
+ *
+ * @param <T> type of elements of this data set
+ */
+@Audience(Audience.Type.CLIENT)
+public class Dataset<T> implements PValue {
+
+  public static <T> Dataset<T> of(PCollection<T> pCollection) {
+    return new Dataset<>(pCollection, null);
+  }
+
+  public static <T> Dataset<T> of(PCollection<T> pCollection, Operator producer) {
+    return new Dataset<>(pCollection, producer);
+  }
+
+  private final PCollection<T> pCollection;
+  @Nullable private final Operator producer;
+
+  private Dataset(PCollection<T> pCollection, @Nullable Operator producer) {
+    this.pCollection = pCollection;
+    this.producer = producer;
+  }
+
+  @Override
+  public String getName() {
+    return pCollection.getName();
+  }
+
+  @Deprecated
+  @Override
+  public Map<TupleTag<?>, PValue> expand() {
+    return null;
+  }
+
+  @Override
+  public void finishSpecifying(PInput upstreamInput, PTransform<?, ?> upstreamTransform) {
+    pCollection.finishSpecifying(upstreamInput, upstreamTransform);
+  }
+
+  @Override
+  public Pipeline getPipeline() {
+    return pCollection.getPipeline();
+  }
+
+  @Override
+  public void finishSpecifyingOutput(
+      String transformName, PInput input, PTransform<?, ?> transform) {
+    pCollection.finishSpecifyingOutput(transformName, input, transform);
+  }
+
+  /**
+   * Get underlying {@link PCollection}.
+   *
+   * @return pCollection
+   */
+  public PCollection<T> getPCollection() {
+    return pCollection;
+  }
+
+  public Optional<Operator> getProducer() {
+    return Optional.ofNullable(producer);
+  }
+
+  public TypeDescriptor<T> getTypeDescriptor() {
+    return pCollection.getTypeDescriptor();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java
new file mode 100644
index 00000000000..c37d9b7387a
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * {@link org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset} and its
+ * implementations.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.dataset;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java
new file mode 100644
index 00000000000..e641a009d46
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Function of two arguments.
+ *
+ * @param <LeftT> the type of the left operand
+ * @param <RightT> the type of the right operand
+ * @param <OutputT> the type of the result of the function application
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface BinaryFunction<LeftT, RightT, OutputT> extends Serializable {
+
+  /**
+   * Applies this function to the given arguments.
+   *
+   * @param left the "left" parameter of the operation
+   * @param right the "right" parameter of the operation
+   * @return the result of applying "left" and "right" to an operation
+   */
+  OutputT apply(LeftT left, RightT right);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java
new file mode 100644
index 00000000000..3f1b28bb1a7
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+
+/** Functor of two arguments. */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface BinaryFunctor<LeftT, RightT, OutputT> extends Serializable {
+
+  void apply(LeftT left, RightT right, Collector<OutputT> context);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java
new file mode 100644
index 00000000000..656b7dd3895
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Function of two arguments of the same type. */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface CombinableBinaryFunction<T> extends SameArgsBinaryFunction<T, T> {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java
new file mode 100644
index 00000000000..41a6ae98f4d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java
@@ -0,0 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Function reducing stream of elements into single one of the same type. The applied function has
+ * to be commutative associative.
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface CombinableReduceFunction<T> extends ReduceFunction<T, T> {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CompositeUnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CompositeUnaryFunction.java
new file mode 100644
index 00000000000..b5e086ae791
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CompositeUnaryFunction.java
@@ -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.beam.sdk.extensions.euphoria.core.client.functional;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** A function that is composition of two unary functions. */
+@Audience(Audience.Type.CLIENT)
+public class CompositeUnaryFunction<InputT, OutputT, X> implements UnaryFunction<InputT, OutputT> {
+
+  private final UnaryFunction<InputT, X> first;
+  private final UnaryFunction<X, OutputT> second;
+
+  private CompositeUnaryFunction(UnaryFunction<InputT, X> first, UnaryFunction<X, OutputT> second) {
+    this.first = first;
+    this.second = second;
+  }
+
+  public static <InputT, OutputT, X> CompositeUnaryFunction<InputT, OutputT, X> of(
+      UnaryFunction<InputT, X> first, UnaryFunction<X, OutputT> second) {
+    return new CompositeUnaryFunction<>(first, second);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public OutputT apply(InputT what) {
+    return second.apply(first.apply(what));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java
new file mode 100644
index 00000000000..6d426ea5856
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java
@@ -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.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** A consumer of given type. */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface Consumer<T> extends Serializable {
+
+  /**
+   * Consume given element.
+   *
+   * @param element the element to be consumed.
+   */
+  void accept(T element);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java
new file mode 100644
index 00000000000..1a9e1d1520b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** @param <InputT> type of input to extract timestamp from */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface ExtractEventTime<InputT> extends Serializable {
+
+  /**
+   * Extracts event time (in millis since epoch) of the given element.
+   *
+   * @param elem input element
+   * @return the extracted event time
+   */
+  long extractTimestamp(InputT elem);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java
new file mode 100644
index 00000000000..f1ed706dd93
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java
@@ -0,0 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.util.stream.Stream;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Reduce function reducing iterable of elements into single element (of possibly different type).
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface ReduceFunction<InputT, OutputT> extends UnaryFunction<Stream<InputT>, OutputT> {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java
new file mode 100644
index 00000000000..188ca39daa8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.util.stream.Stream;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+
+/**
+ * Reduce function reducing iterable of elements into multiple elements (of possibly different
+ * type).
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface ReduceFunctor<InputT, OutputT> extends UnaryFunctor<Stream<InputT>, OutputT> {
+
+  /**
+   * Create reduce functor from combinable function
+   *
+   * @param combinableFunction combinable function
+   * @param <V> value type
+   * @return reduce functor
+   */
+  static <V> ReduceFunctor<V, V> of(CombinableReduceFunction<V> combinableFunction) {
+
+    return new ReduceFunctor<V, V>() {
+
+      @Override
+      public boolean isCombinable() {
+        return true;
+      }
+
+      @Override
+      public void apply(Stream<V> elem, Collector<V> context) {
+        context.collect(combinableFunction.apply(elem));
+      }
+    };
+  }
+
+  /**
+   * Is this a commutative associative function with single final output?
+   *
+   * @return {@code true} if this is combinable function. NOTE: user code should not need to
+   *     override this
+   */
+  default boolean isCombinable() {
+    return false;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java
new file mode 100644
index 00000000000..adcdfbff3d7
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Function of two arguments of the same type. */
+@Audience(Audience.Type.INTERNAL)
+@FunctionalInterface
+public interface SameArgsBinaryFunction<InputT, OutputT>
+    extends BinaryFunction<InputT, InputT, OutputT> {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Supplier.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Supplier.java
new file mode 100644
index 00000000000..b7c3f51baf6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Supplier.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+
+/**
+ * Represents a serializable supplier of some object.
+ *
+ * <p>This is a <a href="package-summary.html">functional interface</a> whose functional method is
+ * {@link #get()}.
+ *
+ * @param <T> the type of results supplied by this supplier
+ */
+@FunctionalInterface
+public interface Supplier<T> extends Serializable {
+
+  /**
+   * Gets a supplied object.
+   *
+   * @return a result
+   */
+  T get();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java
new file mode 100644
index 00000000000..e70009b2bba
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Function taking three arguments. */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface TernaryFunction<FirstT, SecondT, ThirdT, OutputT> extends Serializable {
+
+  OutputT apply(FirstT first, SecondT second, ThirdT third);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java
new file mode 100644
index 00000000000..3940706a693
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java
@@ -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.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Function of single argument.
+ *
+ * @param <InputT> the type of the element processed
+ * @param <OutputT> the type of the result applying element to the function
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface UnaryFunction<InputT, OutputT> extends Serializable {
+
+  /**
+   * Return the result of this function.
+   *
+   * @param what the element applied to the function
+   * @return the result of the function application
+   */
+  OutputT apply(InputT what);
+
+  /**
+   * Returns a {@link UnaryFunction} that always returns its input argument.
+   *
+   * @param <T> the type of the input and output objects to the function
+   * @return a function that always returns its input argument
+   */
+  static <T> UnaryFunction<T, T> identity() {
+    return t -> t;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java
new file mode 100644
index 00000000000..ae39e66349d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context;
+
+/**
+ * Function of single argument with access to Euphoria environment via context.
+ *
+ * @param <InputT> the type of the element processed
+ * @param <OutputT> the type of the result applying element to the function
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface UnaryFunctionEnv<InputT, OutputT> extends Serializable {
+
+  /**
+   * Applies function to given element.
+   *
+   * @param what The element applied to the function
+   * @param context Provides access to the environment.
+   * @return the result of the function application
+   */
+  OutputT apply(InputT what, Context context);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java
new file mode 100644
index 00000000000..d239c9e2a86
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+
+/**
+ * Functor of single argument. Functor can produce zero or more elements in return to a call, for
+ * which it uses a collector.
+ */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface UnaryFunctor<InputT, OutputT> extends Serializable {
+
+  /**
+   * Applies function to given element.
+   *
+   * @param elem Input element.
+   * @param collector Collector to emit results.
+   */
+  void apply(InputT elem, Collector<OutputT> collector);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java
new file mode 100644
index 00000000000..e2115ebc5dc
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Binary function of X returning boolean. */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface UnaryPredicate<T> extends UnaryFunction<T, Boolean> {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java
new file mode 100644
index 00000000000..8e2a7c9d996
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Function taking zero arguments. */
+@Audience(Audience.Type.CLIENT)
+@FunctionalInterface
+public interface VoidFunction<T> extends Serializable {
+
+  T apply();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java
new file mode 100644
index 00000000000..f1d76b06f68
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Collection of user-defined functions. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.functional;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Collector.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Collector.java
new file mode 100644
index 00000000000..f990ff43af0
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Collector.java
@@ -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.beam.sdk.extensions.euphoria.core.client.io;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Extends {@link Environment} with write capability. Used in user defined functors.
+ *
+ * @param <T> the type of elements collected through this context
+ */
+@Audience(Audience.Type.CLIENT)
+public interface Collector<T> extends Environment {
+
+  /**
+   * Collects the given element to the output of this context.
+   *
+   * @param elem the element to collect
+   */
+  void collect(T elem);
+
+  /**
+   * Returns {@link Context} view of the collector. Since {@link Collector} usually share the same
+   * methods as {@link Context} it can be safely casted.
+   *
+   * @return this instance as a context class
+   */
+  Context asContext();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java
new file mode 100644
index 00000000000..8ea8801d8b4
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.io;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Used in user defined functions to access environment methods. */
+@Audience(Audience.Type.CLIENT)
+public interface Context extends Environment {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Environment.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Environment.java
new file mode 100644
index 00000000000..766b7f2f344
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Environment.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.io;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+
+/** Defines basic methods available in user defined functions. */
+@Audience(Audience.Type.CLIENT)
+public interface Environment {
+
+  // ---------------- Aggregator related methods ------------
+
+  /**
+   * Get an existing instance of a counter or create a new one.
+   *
+   * @param name Unique name of the counter.
+   * @return Instance of a counter.
+   */
+  Counter getCounter(String name);
+
+  /**
+   * Get an existing instance of a histogram or create a new one.
+   *
+   * @param name Unique name of the histogram.
+   * @return Instance of a histogram.
+   */
+  Histogram getHistogram(String name);
+
+  /**
+   * Get an existing instance of a timer or create a new one.
+   *
+   * @param name Unique name of the timer.
+   * @return Instance of a timer.
+   */
+  Timer getTimer(String name);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/package-info.java
new file mode 100644
index 00000000000..0130daf8b2c
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Defines abstraction of transformation for reading and writing common storage formats. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.io;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Euphoria.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Euphoria.java
new file mode 100644
index 00000000000..ed4aa8a3a2d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Euphoria.java
@@ -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.beam.sdk.extensions.euphoria.core.client.lib;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.function.Function;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * {@link PTransform} which allows you to build composite transformations in Euphoria API.
+ *
+ * @param <InputT> type of input elements
+ * @param <OutputT> type of output elements
+ */
+public class Euphoria<InputT, OutputT>
+    extends PTransform<PCollection<InputT>, PCollection<OutputT>> {
+
+  private final Function<Dataset<InputT>, Dataset<OutputT>> fn;
+
+  private Euphoria(Function<Dataset<InputT>, Dataset<OutputT>> fn) {
+    this.fn = fn;
+  }
+
+  public static <InputT, OutputT> Euphoria<InputT, OutputT> of(
+      Function<Dataset<InputT>, Dataset<OutputT>> fn) {
+    return new Euphoria<>(requireNonNull(fn));
+  }
+
+  @Override
+  public PCollection<OutputT> expand(PCollection<InputT> input) {
+    return fn.apply(Dataset.of(input)).getPCollection();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java
new file mode 100644
index 00000000000..3bfb0c0d4c6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.lib;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter;
+
+/**
+ * Composite operator using two {@link Filter} operators to split a {@link Dataset} into two subsets
+ * using provided {@link UnaryPredicate}.
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.ZERO, repartitions = 0)
+public class Split {
+
+  static final String DEFAULT_NAME = "Split";
+  static final String POSITIVE_FILTER_SUFFIX = "-positive";
+  static final String NEGATIVE_FILTER_SUFFIX = "-negative";
+
+  public static OfBuilder named(String name) {
+    return new OfBuilder(name);
+  }
+
+  public static <InputT> UsingBuilder<InputT> of(Dataset<InputT> input) {
+    return new UsingBuilder<>(DEFAULT_NAME, input);
+  }
+
+  /** Starting builder. */
+  public static class OfBuilder {
+    private final String name;
+
+    OfBuilder(String name) {
+      this.name = Objects.requireNonNull(name);
+    }
+
+    public <InputT> Split.UsingBuilder<InputT> of(Dataset<InputT> input) {
+      return new Split.UsingBuilder<>(name, input);
+    }
+  }
+
+  /** Builder adding filtering predicate. */
+  public static class UsingBuilder<InputT> {
+    private final String name;
+    private final Dataset<InputT> input;
+
+    UsingBuilder(String name, Dataset<InputT> input) {
+      this.name = Objects.requireNonNull(name);
+      this.input = Objects.requireNonNull(input);
+    }
+
+    public Split.OutputBuilder<InputT> using(UnaryPredicate<InputT> predicate) {
+      return new Split.OutputBuilder<>(name, input, predicate);
+    }
+  }
+
+  /** Last builder in a chain. It concludes this operators creation by calling {@link #output()}. */
+  public static class OutputBuilder<InputT> implements Serializable {
+    private final String name;
+    private final Dataset<InputT> input;
+    private final UnaryPredicate<InputT> predicate;
+
+    OutputBuilder(String name, Dataset<InputT> input, UnaryPredicate<InputT> predicate) {
+      this.name = Objects.requireNonNull(name);
+      this.input = Objects.requireNonNull(input);
+      this.predicate = Objects.requireNonNull(predicate);
+    }
+
+    public Output<InputT> output() {
+      Dataset<InputT> positiveOutput =
+          Filter.named(name + POSITIVE_FILTER_SUFFIX).of(input).by(predicate).output();
+      Dataset<InputT> negativeOutput =
+          Filter.named(name + NEGATIVE_FILTER_SUFFIX)
+              .of(input)
+              .by((UnaryPredicate<InputT>) what -> !predicate.apply(what))
+              .output();
+      return new Output<>(positiveOutput, negativeOutput);
+    }
+  }
+
+  /** KV of positive and negative output as a result of the {@link Split} operator. */
+  public static class Output<T> {
+    private final Dataset<T> positive;
+    private final Dataset<T> negative;
+
+    private Output(Dataset<T> positive, Dataset<T> negative) {
+      this.positive = Objects.requireNonNull(positive);
+      this.negative = Objects.requireNonNull(negative);
+    }
+
+    /** @return positive split result */
+    public Dataset<T> positive() {
+      return positive;
+    }
+
+    /** @return negative split result */
+    public Dataset<T> negative() {
+      return negative;
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java
new file mode 100644
index 00000000000..836760083dc
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** A library of composite operators. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.lib;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java
new file mode 100644
index 00000000000..85d15cdc1da
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * A convenient alias for assignment of event time.
+ *
+ * <p>Can be rewritten as:
+ *
+ * <pre>{@code
+ * Dataset<T> input = ...;
+ * Dataset<T> withStamps = FlatMap.of(input)
+ *    .using(t -> t)
+ *    .eventTimeBy(evt-time-fn)
+ *    .output();
+ * }</pre>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.ZERO, repartitions = 0)
+public class AssignEventTime<InputT> extends Operator<InputT>
+    implements CompositeOperator<InputT, InputT> {
+
+  /**
+   * Starts building a named {@link AssignEventTime} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new {@link AssignEventTime} operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /**
+   * Starts building a nameless {@link AssignEventTime} operator to (re-)assign event time the given
+   * input dataset's elements.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new {@link AssignEventTime} operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> UsingBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /** Builder for the 'of' step from the builder chain. */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> UsingBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for the 'using' step from the builder chain. */
+  public interface UsingBuilder<InputT> {
+
+    /**
+     * @param fn the event time extraction function
+     * @return the next builder to complete the setup
+     * @see FlatMap.EventTimeBuilder#eventTimeBy(ExtractEventTime)
+     */
+    OutputBuilder<InputT> using(ExtractEventTime<InputT> fn);
+  }
+
+  /** Builder for the 'output' step from the builder chain. */
+  public interface OutputBuilder<InputT> extends Builders.Output<InputT> {}
+
+  /**
+   * Last builder in a chain. It concludes this operators creation by calling {@link
+   * #output(OutputHint...)}.
+   */
+  public static class Builder<InputT>
+      implements OfBuilder, UsingBuilder<InputT>, OutputBuilder<InputT> {
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private ExtractEventTime<InputT> eventTimeExtractor;
+
+    private Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T> UsingBuilder<T> of(Dataset<T> input) {
+      @SuppressWarnings("unchecked")
+      final Builder<T> casted = (Builder<T>) this;
+      casted.input = input;
+      return casted;
+    }
+
+    @Override
+    public OutputBuilder<InputT> using(ExtractEventTime<InputT> eventTimeExtractor) {
+      this.eventTimeExtractor = eventTimeExtractor;
+      return this;
+    }
+
+    @Override
+    public Dataset<InputT> output(OutputHint... outputHints) {
+      return OperatorTransform.apply(
+          new AssignEventTime<>(name, eventTimeExtractor, input.getTypeDescriptor()),
+          Collections.singletonList(input));
+    }
+  }
+
+  private final ExtractEventTime<InputT> eventTimeExtractor;
+
+  private AssignEventTime(
+      @Nullable String name,
+      ExtractEventTime<InputT> eventTimeExtractor,
+      @Nullable TypeDescriptor<InputT> outputType) {
+    super(name, outputType);
+    this.eventTimeExtractor = eventTimeExtractor;
+  }
+
+  /**
+   * @return the user defined event time assigner
+   * @see FlatMap#getEventTimeExtractor()
+   */
+  public ExtractEventTime<InputT> getEventTimeExtractor() {
+    return eventTimeExtractor;
+  }
+
+  @Override
+  public Dataset<InputT> expand(List<Dataset<InputT>> inputs) {
+    final Dataset<InputT> input = Iterables.getOnlyElement(inputs);
+    return FlatMap.named(getName().orElse(null))
+        .of(Iterables.getOnlyElement(inputs))
+        .using(
+            (InputT element, Collector<InputT> coll) -> coll.collect(element),
+            input.getTypeDescriptor())
+        .eventTimeBy(getEventTimeExtractor())
+        .output();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CompositeOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CompositeOperator.java
new file mode 100644
index 00000000000..324d8286ac4
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CompositeOperator.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+
+/**
+ * An operator that can be constructed using basic operators.
+ *
+ * @param <InputT> type of input
+ * @param <OutputT> type of output
+ */
+public interface CompositeOperator<InputT, OutputT> {
+
+  /**
+   * Expand input operator to basic operators.
+   *
+   * @param inputs list of input data sets
+   * @return output data set
+   */
+  Dataset<OutputT> expand(List<Dataset<InputT>> inputs);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java
new file mode 100644
index 00000000000..de09586a230
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeUtils;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Operator counting elements with same key.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code keyBy ....................} key extractor function
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.CONSTANT, repartitions = 1)
+public class CountByKey<InputT, KeyT> extends ShuffleOperator<InputT, KeyT, KV<KeyT, Long>>
+    implements CompositeOperator<InputT, KV<KeyT, Long>> {
+
+  /**
+   * Starts building a nameless {@link CountByKey} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input) {
+    return new Builder<>(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link CountByKey} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /** Builder for 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for 'keyBy' step */
+  public interface KeyByBuilder<InputT> extends Builders.KeyBy<InputT> {
+
+    @Override
+    <T> WindowByBuilder<T> keyBy(UnaryFunction<InputT, T> keyExtractor, TypeDescriptor<T> keyType);
+
+    @Override
+    default <T> WindowByBuilder<T> keyBy(UnaryFunction<InputT, T> keyExtractor) {
+      return keyBy(keyExtractor, null);
+    }
+  }
+
+  /** Builder for 'windowBy' step */
+  public interface WindowByBuilder<KeyT>
+      extends Builders.WindowBy<TriggeredByBuilder<KeyT>>,
+          OptionalMethodBuilder<WindowByBuilder<KeyT>, OutputBuilder<KeyT>>,
+          OutputBuilder<KeyT> {
+
+    @Override
+    <W extends BoundedWindow> TriggeredByBuilder<KeyT> windowBy(WindowFn<Object, W> windowing);
+
+    @Override
+    default OutputBuilder<KeyT> applyIf(
+        boolean cond, UnaryFunction<WindowByBuilder<KeyT>, OutputBuilder<KeyT>> fn) {
+      return cond ? requireNonNull(fn).apply(this) : this;
+    }
+  }
+
+  /** Builder for 'triggeredBy' step */
+  public interface TriggeredByBuilder<KeyT>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<KeyT>> {
+
+    @Override
+    AccumulationModeBuilder<KeyT> triggeredBy(Trigger trigger);
+  }
+
+  /** Builder for 'accumulationMode' step */
+  public interface AccumulationModeBuilder<KeyT>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<KeyT>> {
+
+    @Override
+    WindowedOutputBuilder<KeyT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode);
+  }
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<KeyT>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<KeyT>>, OutputBuilder<KeyT> {}
+
+  /** Builder for 'output' step */
+  public interface OutputBuilder<KeyT> extends Builders.Output<KV<KeyT, Long>> {}
+
+  /**
+   * Builder for CountByKey operator.
+   *
+   * @param <InputT> type of input
+   * @param <KeyT> type of key
+   */
+  private static class Builder<InputT, KeyT>
+      implements OfBuilder,
+          KeyByBuilder<InputT>,
+          WindowByBuilder<KeyT>,
+          TriggeredByBuilder<KeyT>,
+          AccumulationModeBuilder<KeyT>,
+          WindowedOutputBuilder<KeyT>,
+          OutputBuilder<KeyT> {
+
+    private final WindowBuilder<InputT> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryFunction<InputT, KeyT> keyExtractor;
+    @Nullable private TypeDescriptor<KeyT> keyType;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> KeyByBuilder<T> of(Dataset<T> input) {
+      this.input = (Dataset<InputT>) requireNonNull(input);
+      return (KeyByBuilder) this;
+    }
+
+    @Override
+    public <T> WindowByBuilder<T> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T> casted = (Builder<InputT, T>) this;
+      casted.keyExtractor = requireNonNull(keyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <W extends BoundedWindow> TriggeredByBuilder<KeyT> windowBy(
+        WindowFn<Object, W> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<KeyT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withAllowedLateness(Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withTimestampCombiner(TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    public Dataset<KV<KeyT, Long>> output(OutputHint... outputHints) {
+      final CountByKey<InputT, KeyT> rbk =
+          new CountByKey<>(
+              name,
+              keyExtractor,
+              keyType,
+              windowBuilder.getWindow().orElse(null),
+              TypeUtils.keyValues(
+                  TypeAwares.orObjects(Optional.ofNullable(keyType)), TypeDescriptors.longs()));
+      return OperatorTransform.apply(rbk, Collections.singletonList(input));
+    }
+  }
+
+  private CountByKey(
+      @Nullable String name,
+      UnaryFunction<InputT, KeyT> keyExtractor,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      @Nullable Window<InputT> window,
+      TypeDescriptor<KV<KeyT, Long>> outputType) {
+    super(name, outputType, keyExtractor, keyType, window);
+  }
+
+  @Override
+  public Dataset<KV<KeyT, Long>> expand(List<Dataset<InputT>> inputs) {
+    return ReduceByKey.named(getName().orElse(null))
+        .of(Iterables.getOnlyElement(inputs))
+        .keyBy(getKeyExtractor())
+        .valueBy(v -> 1L, TypeDescriptors.longs())
+        .combineBy(Sums.ofLongs())
+        .applyIf(
+            getWindow().isPresent(),
+            builder -> {
+              @SuppressWarnings("unchecked")
+              final ReduceByKey.WindowByInternalBuilder<InputT, KeyT, Long> casted =
+                  (ReduceByKey.WindowByInternalBuilder) builder;
+              return casted.windowBy(
+                  getWindow()
+                      .orElseThrow(
+                          () ->
+                              new IllegalStateException(
+                                  "Unable to resolve windowing for CountByKey expansion.")));
+            })
+        .output();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java
new file mode 100644
index 00000000000..b857f69c22d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Operator outputting distinct (based on {@link Object#equals}) elements.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code [mapped] .................} compare objects retrieved by this {@link UnaryFunction}
+ *       instead of raw input elements
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Recommended(
+  reason =
+      "Might be useful to override the default "
+          + "implementation because of performance reasons"
+          + "(e.g. using bloom filters), which might reduce the space complexity",
+  state = StateComplexity.CONSTANT,
+  repartitions = 1
+)
+public class Distinct<InputT, OutputT> extends ShuffleOperator<InputT, OutputT, OutputT>
+    implements CompositeOperator<InputT, OutputT> {
+
+  /**
+   * Starts building a nameless {@link Distinct} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> MappedBuilder<InputT, InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link Distinct} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> MappedBuilder<InputT, InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for the 'mapped' step */
+  public interface MappedBuilder<InputT, OutputT> extends WindowByBuilder<OutputT> {
+
+    /**
+     * Optionally specifies a function to transform the input elements into another type among which
+     * to find the distincts.
+     *
+     * <p>This is, while windowing will be applied on basis of original input elements, the distinct
+     * operator will be carried out on the transformed elements.
+     *
+     * @param <T> the type of the transformed elements
+     * @param mapper a transform function applied to input element
+     * @return the next builder to complete the setup of the {@link Distinct} operator
+     */
+    default <T> WindowByBuilder<T> mapped(UnaryFunction<InputT, T> mapper) {
+      return mapped(mapper, null);
+    }
+
+    <T> WindowByBuilder<T> mapped(
+        UnaryFunction<InputT, T> mapper, @Nullable TypeDescriptor<T> outputType);
+  }
+
+  /** Builder for the 'windowBy' step */
+  public interface WindowByBuilder<OutputT>
+      extends Builders.WindowBy<TriggerByBuilder<OutputT>>,
+          OptionalMethodBuilder<WindowByBuilder<OutputT>, OutputBuilder<OutputT>>,
+          OutputBuilder<OutputT> {
+
+    @Override
+    <T extends BoundedWindow> TriggerByBuilder<OutputT> windowBy(WindowFn<Object, T> windowing);
+
+    @Override
+    default OutputBuilder<OutputT> applyIf(
+        boolean cond, UnaryFunction<WindowByBuilder<OutputT>, OutputBuilder<OutputT>> fn) {
+      return cond ? requireNonNull(fn).apply(this) : this;
+    }
+  }
+
+  /** Builder for the 'triggeredBy' step */
+  public interface TriggerByBuilder<OutputT>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<OutputT>> {
+
+    @Override
+    AccumulationModeBuilder<OutputT> triggeredBy(Trigger trigger);
+  }
+
+  /** Builder for the 'accumulationMode' step */
+  public interface AccumulationModeBuilder<OutputT>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<OutputT>> {
+
+    @Override
+    WindowedOutputBuilder<OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode);
+  }
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<OutputT>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<OutputT>>, OutputBuilder<OutputT> {}
+
+  /** Builder for the 'output' step */
+  public interface OutputBuilder<OutputT> extends Builders.Output<OutputT> {}
+
+  private static class Builder<InputT, OutputT>
+      implements OfBuilder,
+          MappedBuilder<InputT, OutputT>,
+          WindowByBuilder<OutputT>,
+          TriggerByBuilder<OutputT>,
+          AccumulationModeBuilder<OutputT>,
+          WindowedOutputBuilder<OutputT>,
+          OutputBuilder<OutputT> {
+
+    private final WindowBuilder<InputT> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    @Nullable private UnaryFunction<InputT, OutputT> mapper;
+    @Nullable private TypeDescriptor<OutputT> outputType;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T> MappedBuilder<T, T> of(Dataset<T> input) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, T> casted = (Builder) this;
+      casted.input = requireNonNull(input);
+      return casted;
+    }
+
+    @Override
+    public <T> WindowByBuilder<T> mapped(
+        UnaryFunction<InputT, T> mapper, @Nullable TypeDescriptor<T> outputType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T> casted = (Builder) this;
+      casted.mapper = requireNonNull(mapper);
+      casted.outputType = outputType;
+      return casted;
+    }
+
+    @Override
+    public <T extends BoundedWindow> TriggerByBuilder<OutputT> windowBy(
+        WindowFn<Object, T> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<OutputT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withAllowedLateness(Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withTimestampCombiner(
+        TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Dataset<OutputT> output(OutputHint... outputHints) {
+      if (mapper == null) {
+        this.mapper = (UnaryFunction) UnaryFunction.identity();
+      }
+      final Distinct<InputT, OutputT> distinct =
+          new Distinct<>(name, mapper, outputType, windowBuilder.getWindow().orElse(null));
+      return OperatorTransform.apply(distinct, Collections.singletonList(input));
+    }
+  }
+
+  private Distinct(
+      @Nullable String name,
+      UnaryFunction<InputT, OutputT> mapper,
+      @Nullable TypeDescriptor<OutputT> outputType,
+      @Nullable Window<InputT> window) {
+    super(name, outputType, mapper, outputType, window);
+  }
+
+  @Override
+  public Dataset<OutputT> expand(List<Dataset<InputT>> inputs) {
+    final Dataset<KV<OutputT, Void>> distinct =
+        ReduceByKey.named(getName().orElse(null))
+            .of(Iterables.getOnlyElement(inputs))
+            .keyBy(getKeyExtractor())
+            .valueBy(e -> null, TypeDescriptors.nulls())
+            .combineBy(e -> null)
+            .applyIf(
+                getWindow().isPresent(),
+                builder -> {
+                  @SuppressWarnings("unchecked")
+                  final ReduceByKey.WindowByInternalBuilder<InputT, OutputT, Void> casted =
+                      (ReduceByKey.WindowByInternalBuilder) builder;
+                  return casted.windowBy(
+                      getWindow()
+                          .orElseThrow(
+                              () ->
+                                  new IllegalStateException(
+                                      "Unable to resolve windowing for Distinct expansion.")));
+                })
+            .output();
+    return MapElements.named("extract-keys")
+        .of(distinct)
+        .using(KV::getKey, getKeyType().orElse(null))
+        .output();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java
new file mode 100644
index 00000000000..018d28d2394
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Operator performing a filter operation.
+ *
+ * <p>Output elements that pass given condition.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code by .......................} apply {@link UnaryPredicate} to input elements
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.ZERO, repartitions = 0)
+public class Filter<InputT> extends Operator<InputT> implements CompositeOperator<InputT, InputT> {
+
+  /**
+   * Starts building a nameless {@link Filter} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> ByBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link Filter} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> ByBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for the 'by' step */
+  public interface ByBuilder<InputT> {
+
+    /**
+     * Specifies the function that is capable of input elements filtering.
+     *
+     * @param predicate the function that filters out elements if the return value for the element
+     *     is false
+     * @return the next builder to complete the setup of the operator
+     */
+    Builders.Output<InputT> by(UnaryPredicate<InputT> predicate);
+  }
+
+  public interface OutputBuilder<InputT> extends Builders.Output<InputT> {}
+
+  private static class Builder<InputT>
+      implements OfBuilder, ByBuilder<InputT>, OutputBuilder<InputT> {
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryPredicate<InputT> predicate;
+
+    private Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T> ByBuilder<T> of(Dataset<T> input) {
+      @SuppressWarnings("unchecked")
+      final Builder<T> casted = (Builder) this;
+      casted.input = requireNonNull(input);
+      return casted;
+    }
+
+    @Override
+    public Builders.Output<InputT> by(UnaryPredicate<InputT> predicate) {
+      this.predicate = requireNonNull(predicate);
+      return this;
+    }
+
+    @Override
+    public Dataset<InputT> output(OutputHint... outputHints) {
+      final Filter<InputT> filter = new Filter<>(name, predicate, input.getTypeDescriptor());
+      return OperatorTransform.apply(filter, Collections.singletonList(input));
+    }
+  }
+
+  private final UnaryPredicate<InputT> predicate;
+
+  private Filter(
+      @Nullable String name,
+      UnaryPredicate<InputT> predicate,
+      @Nullable TypeDescriptor<InputT> outputType) {
+    super(name, outputType);
+    this.predicate = predicate;
+  }
+
+  public UnaryPredicate<InputT> getPredicate() {
+    return predicate;
+  }
+
+  @Override
+  public Dataset<InputT> expand(List<Dataset<InputT>> inputs) {
+    return FlatMap.named(getName().orElse(null))
+        .of(Iterables.getOnlyElement(inputs))
+        .using(
+            (InputT element, Collector<InputT> collector) -> {
+              if (getPredicate().apply(element)) {
+                collector.collect(element);
+              }
+            },
+            getOutputType().orElse(null))
+        .output();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java
new file mode 100644
index 00000000000..412ad596147
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Collections;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * A transformation of a dataset from one type into another allowing user code to generate zero,
+ * one, or many output elements for a given input element.
+ *
+ * <p>The user supplied map function is supposed to be stateless. It is fed items from the input in
+ * no specified order and the results of the map function are "flattened" to the output (equally in
+ * no specified order.)
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * Dataset<String> strings = ...;
+ * Dataset<Integer> ints =
+ *        FlatMap.named("TO-INT")
+ *           .of(strings)
+ *           .using((String s, Context<String> c) -> {
+ *             try {
+ *               int i = Integer.parseInt(s);
+ *               c.collect(i);
+ *             } catch (NumberFormatException e) {
+ *               // ~ ignore the input if we failed to parse it
+ *             }
+ *           })
+ *           .output();
+ * }</pre>
+ *
+ * <p>The above example tries to parse incoming strings as integers, silently skipping those which
+ * cannot be successfully converted. While {@link Collector#collect(Object)} has been used only once
+ * here, a {@link FlatMap} operator is free to invoke it multiple times or not at all to generate
+ * that many elements to the output dataset.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code using ....................} apply {@link UnaryFunctor} to input elements
+ *   <li>{@code [eventTimeBy] ............} change event time characteristic of output elements
+ *       using {@link ExtractEventTime}
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Basic(state = StateComplexity.ZERO, repartitions = 0)
+public class FlatMap<InputT, OutputT> extends Operator<OutputT>
+    implements TypeAware.Output<OutputT> {
+
+  /**
+   * Starts building a nameless {@link FlatMap} operator to transform the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be transformed
+   * @return a builder to complete the setup of the new {@link FlatMap} operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> UsingBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link FlatMap} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new {@link FlatMap} operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder<>(name);
+  }
+
+  // ------------- Builders chain
+
+  /** Builder exposing {@link #of(Dataset)} method. */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> UsingBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /**
+   * A builder which allows user to determine {@link FlatMap FlatMap's} {@link UnaryFunctor
+   * functor}.
+   *
+   * @param <InputT> Input elements type parameter.
+   */
+  public interface UsingBuilder<InputT> {
+
+    /**
+     * Specifies the user defined map function by which to transform the final operator's input
+     * dataset.
+     *
+     * @param <OutputT> the type of elements the user defined map function will produce to the
+     *     output dataset
+     * @param functor the user defined map function
+     * @return the next builder to complete the setup of the {@link FlatMap} operator
+     */
+    <OutputT> EventTimeBuilder<InputT, OutputT> using(UnaryFunctor<InputT, OutputT> functor);
+
+    <OutputT> EventTimeBuilder<InputT, OutputT> using(
+        UnaryFunctor<InputT, OutputT> functor, TypeDescriptor<OutputT> outputTypeDescriptor);
+  }
+
+  /**
+   * Builder allowing user to specify how event time is associated with input elements.
+   *
+   * @param <InputT> input elements type
+   * @param <OutputT> output elements type
+   */
+  public interface EventTimeBuilder<InputT, OutputT> extends Builders.Output<OutputT> {
+
+    /**
+     * Specifies a function to derive the input element's event time. Processing of the input stream
+     * continues then to proceed with this event time.
+     *
+     * @param eventTimeFn the event time extraction function
+     * @return the next builder to complete the setup of the {@link FlatMap} operator
+     */
+    OutputBuilder<OutputT> eventTimeBy(ExtractEventTime<InputT> eventTimeFn);
+  }
+
+  /**
+   * Last builder in a chain. It concludes this operators creation by calling {@link
+   * #output(OutputHint...)}.
+   */
+  public interface OutputBuilder<OutputT> extends Builders.Output<OutputT> {
+
+    @Override
+    Dataset<OutputT> output(OutputHint... outputHints);
+  }
+
+  /** Builder of {@link FlatMap}. */
+  public static class Builder<InputT, OutputT>
+      implements OfBuilder,
+          UsingBuilder<InputT>,
+          EventTimeBuilder<InputT, OutputT>,
+          OutputBuilder<OutputT> {
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryFunctor<InputT, OutputT> functor;
+    @Nullable private TypeDescriptor<OutputT> outputType;
+    @Nullable private ExtractEventTime<InputT> evtTimeFn;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <InputLocalT> UsingBuilder<InputLocalT> of(Dataset<InputLocalT> input) {
+      @SuppressWarnings("unchecked")
+      Builder<InputLocalT, ?> casted = (Builder) this;
+      casted.input = requireNonNull(input);
+      return casted;
+    }
+
+    @Override
+    public <OutputLocalT> EventTimeBuilder<InputT, OutputLocalT> using(
+        UnaryFunctor<InputT, OutputLocalT> functor) {
+      return using(functor, null);
+    }
+
+    @Override
+    public <OutputLocalT> EventTimeBuilder<InputT, OutputLocalT> using(
+        UnaryFunctor<InputT, OutputLocalT> functor, TypeDescriptor<OutputLocalT> outputType) {
+      @SuppressWarnings("unchecked")
+      Builder<InputT, OutputLocalT> casted = (Builder) this;
+      casted.functor = requireNonNull(functor);
+      casted.outputType = outputType;
+      return casted;
+    }
+
+    @Override
+    public OutputBuilder<OutputT> eventTimeBy(ExtractEventTime<InputT> eventTimeFn) {
+      this.evtTimeFn = requireNonNull(eventTimeFn);
+      return this;
+    }
+
+    @Override
+    public Dataset<OutputT> output(OutputHint... outputHints) {
+      return OperatorTransform.apply(
+          new FlatMap<>(name, functor, outputType, evtTimeFn), Collections.singletonList(input));
+    }
+  }
+
+  private final UnaryFunctor<InputT, OutputT> functor;
+  @Nullable private final ExtractEventTime<InputT> eventTimeFn;
+
+  private FlatMap(
+      @Nullable String name,
+      UnaryFunctor<InputT, OutputT> functor,
+      @Nullable TypeDescriptor<OutputT> outputType,
+      @Nullable ExtractEventTime<InputT> evtTimeFn) {
+    super(name, outputType);
+    this.functor = functor;
+    this.eventTimeFn = evtTimeFn;
+  }
+
+  /**
+   * Retrieves the user defined map function to be applied to this operator's input elements.
+   *
+   * @return the user defined map function; never {@code null}
+   */
+  public UnaryFunctor<InputT, OutputT> getFunctor() {
+    return functor;
+  }
+
+  /**
+   * Retrieves the optional user defined event time assigner.
+   *
+   * @return the user defined event time assigner if specified
+   */
+  public Optional<ExtractEventTime<InputT>> getEventTimeExtractor() {
+    return Optional.ofNullable(eventTimeFn);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java
new file mode 100644
index 00000000000..598c269c298
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join.Type;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Full outer join of two input datasets producing single new dataset.
+ *
+ * <p>When joining two streams, the join has to specify windowing which groups elements from streams
+ * into {@link org.apache.beam.sdk.transforms.windowing.Window}s. The join operation is performed
+ * within same windows produced on left and right side of input {@link Dataset}s.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} left and right input dataset
+ *   <li>{@code by .......................} {@link UnaryFunction}s transforming left and right
+ *       elements into keys
+ *   <li>{@code using ....................} {@link BinaryFunctor} receiving left and right element
+ *       from joined window
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code (output | outputValues) ..} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+public class FullJoin {
+
+  /**
+   * Create builder.
+   *
+   * @param left dataset
+   * @param right dataset
+   * @param <LeftT> type of left dataset
+   * @param <RightT> type of right dataset
+   * @return ByBuilder
+   */
+  public static <LeftT, RightT> ByBuilder<LeftT, RightT> of(
+      Dataset<LeftT> left, Dataset<RightT> right) {
+    return named("FullJoin").of(left, right);
+  }
+
+  /**
+   * Optional setter to give operator name.
+   *
+   * @param name of the operator
+   * @return OfBuilder
+   */
+  public static OfBuilder named(String name) {
+    return new Builder<>(name);
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder {
+
+    <LeftT, RightT> ByBuilder<LeftT, RightT> of(Dataset<LeftT> left, Dataset<RightT> right);
+  }
+
+  /** Builder for the 'by' step */
+  public interface ByBuilder<LeftT, RightT> {
+
+    <KeyT> UsingBuilder<LeftT, RightT, KeyT> by(
+        UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+        UnaryFunction<RightT, KeyT> rightKeyExtractor,
+        @Nullable TypeDescriptor<KeyT> keyType);
+
+    default <KeyT> UsingBuilder<LeftT, RightT, KeyT> by(
+        UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+        UnaryFunction<RightT, KeyT> rightKeyExtractor) {
+      return by(leftKeyExtractor, rightKeyExtractor, null);
+    }
+  }
+
+  /** Builder for the 'using' step */
+  public interface UsingBuilder<LeftT, RightT, KeyT> {
+
+    <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<Optional<LeftT>, Optional<RightT>, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputType);
+
+    default <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<Optional<LeftT>, Optional<RightT>, OutputT> joinFunc) {
+      return using(joinFunc, null);
+    }
+  }
+
+  private static class Builder<LeftT, RightT, KeyT>
+      implements OfBuilder, ByBuilder<LeftT, RightT>, UsingBuilder<LeftT, RightT, KeyT> {
+
+    private final String name;
+    private Dataset<LeftT> left;
+    private Dataset<RightT> right;
+    private UnaryFunction<LeftT, KeyT> leftKeyExtractor;
+    private UnaryFunction<RightT, KeyT> rightKeyExtractor;
+    @Nullable TypeDescriptor<KeyT> keyType;
+
+    private Builder(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T, S> ByBuilder<T, S> of(Dataset<T> left, Dataset<S> right) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, S, ?> casted = (Builder) this;
+      casted.left = requireNonNull(left);
+      casted.right = requireNonNull(right);
+      return casted;
+    }
+
+    @Override
+    public <T> UsingBuilder<LeftT, RightT, T> by(
+        UnaryFunction<LeftT, T> leftKeyExtractor,
+        UnaryFunction<RightT, T> rightKeyExtractor,
+        @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<LeftT, RightT, T> casted = (Builder) this;
+      casted.leftKeyExtractor = requireNonNull(leftKeyExtractor);
+      casted.rightKeyExtractor = requireNonNull(rightKeyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<Optional<LeftT>, Optional<RightT>, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputType) {
+      return new Join.Builder<>(name, Type.FULL)
+          .of(left, right)
+          .by(leftKeyExtractor, rightKeyExtractor, keyType)
+          .using(
+              (LeftT l, RightT r, Collector<OutputT> c) ->
+                  joinFunc.apply(Optional.ofNullable(l), Optional.ofNullable(r), c),
+              outputType);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java
new file mode 100644
index 00000000000..c2f911b3579
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java
@@ -0,0 +1,344 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Inner join of two datasets by given key producing single new dataset.
+ *
+ * <p>When joining two streams, the join has to specify windowing which groups elements from streams
+ * into {@link Window}s. The join operation is performed within same windows produced on left and
+ * right side of input {@link Dataset}s.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} left and right input dataset
+ *   <li>{@code by .......................} {@link UnaryFunction}s transforming left and right
+ *       elements into keys
+ *   <li>{@code using ....................} {@link BinaryFunctor} receiving left and right element
+ *       from joined window
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code (output | outputValues) ..} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Recommended(
+  reason =
+      "Might be useful to override because of performance reasons in a "
+          + "specific join types (e.g. sort join), which might reduce the space "
+          + "complexity",
+  state = StateComplexity.LINEAR,
+  repartitions = 1
+)
+public class Join<LeftT, RightT, KeyT, OutputT>
+    extends ShuffleOperator<Object, KeyT, KV<KeyT, OutputT>> {
+
+  public static <LeftT, RightT> ByBuilder<LeftT, RightT> of(
+      Dataset<LeftT> left, Dataset<RightT> right) {
+    return named(null).of(left, right);
+  }
+
+  /**
+   * Name of join operator.
+   *
+   * @param name of operator
+   * @return OfBuilder
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder<>(name, Type.INNER);
+  }
+
+  /** Type of join. */
+  public enum Type {
+    INNER,
+    LEFT,
+    RIGHT,
+    FULL
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder {
+
+    <LeftT, RightT> ByBuilder<LeftT, RightT> of(Dataset<LeftT> left, Dataset<RightT> right);
+  }
+
+  /** Builder for the 'by' step */
+  public interface ByBuilder<LeftT, RightT> {
+
+    <K> UsingBuilder<LeftT, RightT, K> by(
+        UnaryFunction<LeftT, K> leftKeyExtractor,
+        UnaryFunction<RightT, K> rightKeyExtractor,
+        @Nullable TypeDescriptor<K> keyType);
+
+    default <T> UsingBuilder<LeftT, RightT, T> by(
+        UnaryFunction<LeftT, T> leftKeyExtractor, UnaryFunction<RightT, T> rightKeyExtractor) {
+      return by(leftKeyExtractor, rightKeyExtractor, null);
+    }
+  }
+
+  /** Builder for the 'using' step */
+  public interface UsingBuilder<LeftT, RightT, KeyT> {
+
+    <OutputT> WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<LeftT, RightT, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputTypeDescriptor);
+
+    default <OutputT> WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<LeftT, RightT, OutputT> joinFunc) {
+      return using(joinFunc, null);
+    }
+  }
+
+  /** Builder for the 'windowBy' step */
+  public interface WindowByBuilder<KeyT, OutputT>
+      extends OptionalMethodBuilder<WindowByBuilder<KeyT, OutputT>, OutputBuilder<KeyT, OutputT>>,
+          Builders.WindowBy<TriggeredByBuilder<KeyT, OutputT>>,
+          OutputBuilder<KeyT, OutputT> {
+
+    @Override
+    default OutputBuilder<KeyT, OutputT> applyIf(
+        boolean cond,
+        UnaryFunction<WindowByBuilder<KeyT, OutputT>, OutputBuilder<KeyT, OutputT>> fn) {
+      return cond ? requireNonNull(fn).apply(this) : this;
+    }
+  }
+
+  /** Builder for the 'triggeredBy' step */
+  public interface TriggeredByBuilder<KeyT, OutputT>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<KeyT, OutputT>> {}
+
+  /** Builder for the 'accumulatorMode' step */
+  public interface AccumulationModeBuilder<KeyT, OutputT>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<KeyT, OutputT>> {}
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<KeyT, OutputT>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<KeyT, OutputT>>,
+          OutputBuilder<KeyT, OutputT> {}
+
+  /**
+   * Last builder in a chain. It concludes this operators creation by calling {@link
+   * #output(OutputHint...)}.
+   */
+  public interface OutputBuilder<KeyT, OutputT>
+      extends Builders.Output<KV<KeyT, OutputT>>, Builders.OutputValues<KeyT, OutputT> {}
+
+  /** Parameters of this operator used in builders. */
+  static class Builder<LeftT, RightT, KeyT, OutputT>
+      implements OfBuilder,
+          ByBuilder<LeftT, RightT>,
+          UsingBuilder<LeftT, RightT, KeyT>,
+          WindowByBuilder<KeyT, OutputT>,
+          TriggeredByBuilder<KeyT, OutputT>,
+          AccumulationModeBuilder<KeyT, OutputT>,
+          WindowedOutputBuilder<KeyT, OutputT>,
+          OutputBuilder<KeyT, OutputT> {
+
+    private final WindowBuilder<Object> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private final Type type;
+    private Dataset<LeftT> left;
+    private Dataset<RightT> right;
+    private UnaryFunction<LeftT, KeyT> leftKeyExtractor;
+    private UnaryFunction<RightT, KeyT> rightKeyExtractor;
+    @Nullable private TypeDescriptor<KeyT> keyType;
+    private BinaryFunctor<LeftT, RightT, OutputT> joinFunc;
+    @Nullable private TypeDescriptor<OutputT> outputType;
+
+    Builder(@Nullable String name, Type type) {
+      this.name = name;
+      this.type = type;
+    }
+
+    @Override
+    public <T, S> ByBuilder<T, S> of(Dataset<T> left, Dataset<S> right) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, S, ?, ?> casted = (Builder) this;
+      casted.left = requireNonNull(left);
+      casted.right = requireNonNull(right);
+      return casted;
+    }
+
+    @Override
+    public <T> UsingBuilder<LeftT, RightT, T> by(
+        UnaryFunction<LeftT, T> leftKeyExtractor,
+        UnaryFunction<RightT, T> rightKeyExtractor,
+        @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<LeftT, RightT, T, ?> casted = (Builder) this;
+      casted.leftKeyExtractor = leftKeyExtractor;
+      casted.rightKeyExtractor = rightKeyExtractor;
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <T> WindowByBuilder<KeyT, T> using(
+        BinaryFunctor<LeftT, RightT, T> joinFunc, @Nullable TypeDescriptor<T> outputType) {
+      @SuppressWarnings("unchecked")
+      final Builder<LeftT, RightT, KeyT, T> casted = (Builder) this;
+      casted.joinFunc = requireNonNull(joinFunc);
+      casted.outputType = outputType;
+      return casted;
+    }
+
+    @Override
+    public <W extends BoundedWindow> TriggeredByBuilder<KeyT, OutputT> windowBy(
+        WindowFn<Object, W> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<KeyT, OutputT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withAllowedLateness(Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withTimestampCombiner(
+        TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    public Dataset<KV<KeyT, OutputT>> output(OutputHint... outputHints) {
+      final Join<LeftT, RightT, KeyT, OutputT> join =
+          new Join<>(
+              name,
+              type,
+              leftKeyExtractor,
+              rightKeyExtractor,
+              keyType,
+              joinFunc,
+              TypeDescriptors.kvs(
+                  TypeAwares.orObjects(Optional.ofNullable(keyType)),
+                  TypeAwares.orObjects(Optional.ofNullable(outputType))),
+              windowBuilder.getWindow().orElse(null));
+      @SuppressWarnings("unchecked")
+      final List<Dataset<Object>> inputs = Arrays.asList((Dataset) left, (Dataset) right);
+      return OperatorTransform.apply(join, inputs);
+    }
+
+    @Override
+    public Dataset<OutputT> outputValues(OutputHint... outputHints) {
+      return MapElements.named(name != null ? name + "::extract-values" : null)
+          .of(output(outputHints))
+          .using(KV::getValue, outputType)
+          .output(outputHints);
+    }
+  }
+
+  private final Type type;
+  private final UnaryFunction<LeftT, KeyT> leftKeyExtractor;
+  private final UnaryFunction<RightT, KeyT> rightKeyExtractor;
+  private final BinaryFunctor<LeftT, RightT, OutputT> functor;
+
+  private Join(
+      @Nullable String name,
+      Type type,
+      UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+      UnaryFunction<RightT, KeyT> rightKeyExtractor,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      BinaryFunctor<LeftT, RightT, OutputT> functor,
+      @Nullable TypeDescriptor<KV<KeyT, OutputT>> outputType,
+      @Nullable Window<Object> window) {
+    super(name, outputType, null, keyType, window);
+    this.type = type;
+    this.leftKeyExtractor = leftKeyExtractor;
+    this.rightKeyExtractor = rightKeyExtractor;
+    this.functor = functor;
+  }
+
+  public Type getType() {
+    return type;
+  }
+
+  public UnaryFunction<LeftT, KeyT> getLeftKeyExtractor() {
+    return leftKeyExtractor;
+  }
+
+  public UnaryFunction<RightT, KeyT> getRightKeyExtractor() {
+    return rightKeyExtractor;
+  }
+
+  public BinaryFunctor<LeftT, RightT, OutputT> getJoiner() {
+    return functor;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java
new file mode 100644
index 00000000000..845a005d6c5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Left outer join of two input datasets producing single new dataset.
+ *
+ * <p>When joining two streams, the join has to specify windowing which groups elements from streams
+ * into {@link org.apache.beam.sdk.transforms.windowing.Window}s. The join operation is performed
+ * within same windows produced on left and right side of input {@link Dataset}s.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} left and right input dataset
+ *   <li>{@code by .......................} {@link UnaryFunction}s transforming left and right
+ *       elements into keys
+ *   <li>{@code using ....................} {@link BinaryFunctor} receiving left and right element
+ *       from joined window
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code (output | outputValues) ..} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+public class LeftJoin {
+
+  /**
+   * Create builder.
+   *
+   * @param left dataset
+   * @param right dataset
+   * @param <LeftT> type of left dataset
+   * @param <RightT> type of right dataset
+   * @return ByBuilder
+   */
+  public static <LeftT, RightT> ByBuilder<LeftT, RightT> of(
+      Dataset<LeftT> left, Dataset<RightT> right) {
+    return named("LeftJoin").of(left, right);
+  }
+
+  /**
+   * Optional setter to give operator name.
+   *
+   * @param name of the operator
+   * @return OfBuilder
+   */
+  public static OfBuilder named(String name) {
+    return new Builder<>(name);
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder {
+
+    <LeftT, RightT> ByBuilder<LeftT, RightT> of(Dataset<LeftT> left, Dataset<RightT> right);
+  }
+
+  /** Builder for the 'by' step */
+  public interface ByBuilder<LeftT, RightT> {
+
+    <KeyT> UsingBuilder<LeftT, RightT, KeyT> by(
+        UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+        UnaryFunction<RightT, KeyT> rightKeyExtractor,
+        @Nullable TypeDescriptor<KeyT> keyType);
+
+    default <KeyT> UsingBuilder<LeftT, RightT, KeyT> by(
+        UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+        UnaryFunction<RightT, KeyT> rightKeyExtractor) {
+      return by(leftKeyExtractor, rightKeyExtractor, null);
+    }
+  }
+
+  /** Builder for the 'using' step */
+  public interface UsingBuilder<LeftT, RightT, KeyT> {
+
+    <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<LeftT, Optional<RightT>, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputType);
+
+    default <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<LeftT, Optional<RightT>, OutputT> joinFunc) {
+      return using(joinFunc, null);
+    }
+  }
+
+  private static class Builder<LeftT, RightT, KeyT>
+      implements OfBuilder, ByBuilder<LeftT, RightT>, UsingBuilder<LeftT, RightT, KeyT> {
+
+    private final String name;
+    private Dataset<LeftT> left;
+    private Dataset<RightT> right;
+    private UnaryFunction<LeftT, KeyT> leftKeyExtractor;
+    private UnaryFunction<RightT, KeyT> rightKeyExtractor;
+    @Nullable TypeDescriptor<KeyT> keyType;
+
+    private Builder(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T, S> ByBuilder<T, S> of(Dataset<T> left, Dataset<S> right) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, S, ?> casted = (Builder) this;
+      casted.left = requireNonNull(left);
+      casted.right = requireNonNull(right);
+      return casted;
+    }
+
+    @Override
+    public <T> UsingBuilder<LeftT, RightT, T> by(
+        UnaryFunction<LeftT, T> leftKeyExtractor,
+        UnaryFunction<RightT, T> rightKeyExtractor,
+        @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<LeftT, RightT, T> casted = (Builder) this;
+      casted.leftKeyExtractor = requireNonNull(leftKeyExtractor);
+      casted.rightKeyExtractor = requireNonNull(rightKeyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<LeftT, Optional<RightT>, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputType) {
+      return new Join.Builder<>(name, Join.Type.LEFT)
+          .of(left, right)
+          .by(leftKeyExtractor, rightKeyExtractor, keyType)
+          .using(
+              (LeftT l, RightT r, Collector<OutputT> c) ->
+                  joinFunc.apply(l, Optional.ofNullable(r), c),
+              outputType);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java
new file mode 100644
index 00000000000..87c62177efb
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctionEnv;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Simple one-to-one transformation of input elements. It is a special case of {@link FlatMap} with
+ * exactly one output element for every one input element. No context is provided inside the map
+ * function.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code using ....................} apply {@link UnaryFunction} or {@link UnaryFunctionEnv}
+ *       to input elements
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.ZERO, repartitions = 0)
+public class MapElements<InputT, OutputT> extends Operator<OutputT>
+    implements CompositeOperator<InputT, OutputT>, TypeAware.Output<OutputT> {
+
+  /**
+   * Starts building a nameless {@link MapElements} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> UsingBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link MapElements} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder<>(name);
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> UsingBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** MapElements builder which adds mapper to operator under build. */
+  public interface UsingBuilder<InputT> {
+
+    /**
+     * The mapping function that takes input element and outputs the OutputT type element. If you
+     * want use aggregators use rather {@link #using(UnaryFunctionEnv)}.
+     *
+     * @param <OutputT> type of output elements
+     * @param mapper the mapping function
+     * @return the next builder to complete the setup of the {@link MapElements} operator
+     */
+    default <OutputT> OutputBuilder<OutputT> using(UnaryFunction<InputT, OutputT> mapper) {
+      return using(mapper, null);
+    }
+
+    default <OutputT> OutputBuilder<OutputT> using(
+        UnaryFunction<InputT, OutputT> mapper, @Nullable TypeDescriptor<OutputT> outputType) {
+      return using((el, ctx) -> mapper.apply(el), outputType);
+    }
+
+    /**
+     * The mapping function that takes input element and outputs the OutputT type element.
+     *
+     * @param <OutputT> type of output elements
+     * @param mapper the mapping function
+     * @return the next builder to complete the setup of the {@link MapElements} operator
+     */
+    default <OutputT> OutputBuilder<OutputT> using(UnaryFunctionEnv<InputT, OutputT> mapper) {
+      return using(mapper, null);
+    }
+
+    <OutputT> OutputBuilder<OutputT> using(
+        UnaryFunctionEnv<InputT, OutputT> mapper, @Nullable TypeDescriptor<OutputT> outputType);
+  }
+
+  /**
+   * Last builder in a chain. It concludes this operators creation by calling {@link
+   * #output(OutputHint...)}.
+   */
+  public interface OutputBuilder<OutputT> extends Builders.Output<OutputT> {}
+
+  private static class Builder<InputT, OutputT>
+      implements OfBuilder, UsingBuilder<InputT>, OutputBuilder<OutputT> {
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryFunctionEnv<InputT, OutputT> mapper;
+    @Nullable private TypeDescriptor<OutputT> outputType;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T> UsingBuilder<T> of(Dataset<T> input) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, ?> casted = (Builder) this;
+      casted.input = input;
+      return casted;
+    }
+
+    @Override
+    public <T> OutputBuilder<T> using(
+        UnaryFunctionEnv<InputT, T> mapper, @Nullable TypeDescriptor<T> outputType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T> casted = (Builder) this;
+      casted.mapper = mapper;
+      casted.outputType = outputType;
+      return casted;
+    }
+
+    @Override
+    public Dataset<OutputT> output(OutputHint... outputHints) {
+      final MapElements<InputT, OutputT> operator = new MapElements<>(name, mapper, outputType);
+      return OperatorTransform.apply(operator, Collections.singletonList(input));
+    }
+  }
+
+  private final UnaryFunctionEnv<InputT, OutputT> mapper;
+
+  private MapElements(
+      @Nullable String name,
+      UnaryFunctionEnv<InputT, OutputT> mapper,
+      @Nullable TypeDescriptor<OutputT> outputType) {
+    super(name, outputType);
+    this.mapper = mapper;
+  }
+
+  @Override
+  public Dataset<OutputT> expand(List<Dataset<InputT>> inputs) {
+    return FlatMap.named(getName().orElse(null))
+        .of(Iterables.getOnlyElement(inputs))
+        .using(
+            (InputT elem, Collector<OutputT> coll) ->
+                coll.collect(getMapper().apply(elem, coll.asContext())),
+            getOutputType().orElse(null))
+        .output();
+  }
+
+  public UnaryFunctionEnv<InputT, OutputT> getMapper() {
+    return mapper;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java
new file mode 100644
index 00000000000..4da2f744d4f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java
@@ -0,0 +1,507 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Collections;
+import java.util.Optional;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Operator performing state-less aggregation by given reduce function. The reduction is performed
+ * on all extracted values on each key-window.
+ *
+ * <p>If provided function is {@link CombinableReduceFunction} partial reduction is performed before
+ * shuffle. If the function is not combinable all values must be first sent through the network and
+ * the reduction is done afterwards on target machines.
+ *
+ * <p>Custom windowing can be set, otherwise values from input operator are used.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code keyBy ....................} key extractor function
+ *   <li>{@code [valueBy] ................} value extractor function (default: identity)
+ *   <li>{@code (combineBy | reduceBy)....} {@link CombinableReduceFunction} or {@link
+ *       ReduceFunction} for combinable or non-combinable function
+ *   <li>{@code [withSortedValues] .......} use comparator for sorting values prior to being passed
+ *       to {@link ReduceFunction} function (applicable only for non-combinable version)
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code (output | outputValues) ..} build output dataset
+ * </ol>
+ *
+ * @param <InputT> Type of input records
+ * @param <KeyT> Output type of #keyBy method
+ * @param <ValueT> Output type of #valueBy method
+ * @param <OutputT> Type of output value
+ */
+@Audience(Audience.Type.CLIENT)
+@Recommended(
+  reason =
+      "Is very recommended to override because of performance in "
+          + "a specific area of (mostly) batch calculations where combiners "
+          + "can be efficiently used in the executor-specific implementation",
+  state = StateComplexity.CONSTANT_IF_COMBINABLE,
+  repartitions = 1
+)
+public class ReduceByKey<InputT, KeyT, ValueT, OutputT>
+    extends ShuffleOperator<InputT, KeyT, KV<KeyT, OutputT>> implements TypeAware.Value<ValueT> {
+
+  /**
+   * Starts building a nameless {@link ReduceByKey} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link ReduceByKey} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /** Builder for 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for 'keyBy' step */
+  public interface KeyByBuilder<InputT> extends Builders.KeyBy<InputT> {
+
+    @Override
+    <T> ValueByReduceByBuilder<InputT, T, InputT> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, TypeDescriptor<T> keyType);
+
+    @Override
+    default <T> ValueByReduceByBuilder<InputT, T, InputT> keyBy(
+        UnaryFunction<InputT, T> keyExtractor) {
+      return keyBy(keyExtractor, null);
+    }
+  }
+
+  /** Builder for 'reduceBy' step */
+  public interface ReduceByBuilder<KeyT, ValueT> {
+
+    /**
+     * Define a function that reduces all values related to one key into one result object. The
+     * function is not combinable - i.e. partial results cannot be made up before shuffle. To get
+     * better performance use {@link #combineBy} method.
+     *
+     * @param <OutputT> type of output element
+     * @param reducer function that reduces all values into one output object
+     * @return next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    default <OutputT> WithSortedValuesBuilder<KeyT, ValueT, OutputT> reduceBy(
+        ReduceFunction<ValueT, OutputT> reducer) {
+      return reduceBy(
+          (Stream<ValueT> in, Collector<OutputT> ctx) -> ctx.collect(reducer.apply(in)));
+    }
+
+    default <OutputT> WithSortedValuesBuilder<KeyT, ValueT, OutputT> reduceBy(
+        ReduceFunction<ValueT, OutputT> reducer, TypeDescriptor<OutputT> outputType) {
+      return reduceBy(
+          (Stream<ValueT> in, Collector<OutputT> ctx) -> ctx.collect(reducer.apply(in)),
+          outputType);
+    }
+
+    /**
+     * Define a function that reduces all values related to one key into one or more result objects.
+     * The function is not combinable - i.e. partial results cannot be made up before shuffle. To
+     * get better performance use {@link #combineBy} method.
+     *
+     * @param <OutputT> type of output element
+     * @param reducer function that reduces all values into output values
+     * @return next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    default <OutputT> WithSortedValuesBuilder<KeyT, ValueT, OutputT> reduceBy(
+        ReduceFunctor<ValueT, OutputT> reducer) {
+      return reduceBy(reducer, null);
+    }
+
+    <OutputT> WithSortedValuesBuilder<KeyT, ValueT, OutputT> reduceBy(
+        ReduceFunctor<ValueT, OutputT> reducer, @Nullable TypeDescriptor<OutputT> outputType);
+
+    /**
+     * Define a function that reduces all values related to one key into one result object. The
+     * function is combinable (associative and commutative) so it can be used to compute partial
+     * results before shuffle.
+     *
+     * @param reducer function that reduces all values into one output object
+     * @return next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    default WindowByBuilder<KeyT, ValueT> combineBy(CombinableReduceFunction<ValueT> reducer) {
+      return reduceBy(ReduceFunctor.of(reducer));
+    }
+
+    default WindowByBuilder<KeyT, ValueT> combineBy(
+        CombinableReduceFunction<ValueT> reducer, TypeDescriptor<ValueT> outputType) {
+      return reduceBy(ReduceFunctor.of(reducer), outputType);
+    }
+  }
+
+  /** Builder for 'valueBy' / 'reduceBy' step */
+  public interface ValueByReduceByBuilder<InputT, KeyT, ValueT>
+      extends ReduceByBuilder<KeyT, ValueT> {
+
+    /**
+     * Specifies the function to derive a value from the {@link ReduceByKey} operator's input
+     * elements to get reduced by a later supplied reduce function.
+     *
+     * @param <T> the type of the extracted values
+     * @param valueExtractor a user defined function to extract values from the processed input
+     *     dataset's elements for later reduction
+     * @param valueType {@link TypeDescriptor} of value type {@code <V>}
+     * @return the next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    <T> ReduceByBuilder<KeyT, T> valueBy(
+        UnaryFunction<InputT, T> valueExtractor, @Nullable TypeDescriptor<T> valueType);
+
+    default <T> ReduceByBuilder<KeyT, T> valueBy(UnaryFunction<InputT, T> valueExtractor) {
+      return valueBy(valueExtractor, null);
+    }
+  }
+
+  /** Builder for 'withSortedValues' step */
+  public interface WithSortedValuesBuilder<KeyT, ValueT, OutputT>
+      extends WindowByBuilder<KeyT, OutputT> {
+
+    /**
+     * Sort values going to `reduceBy` function by given comparator.
+     *
+     * @param comparator function with contract defined by {@code java.util.Comparator#compare}.
+     * @return next step builder
+     */
+    WindowByBuilder<KeyT, OutputT> withSortedValues(
+        BinaryFunction<ValueT, ValueT, Integer> comparator);
+  }
+
+  /** Internal builder for 'windowBy' step */
+  @Internal
+  public interface WindowByInternalBuilder<InputT, KeyT, OutputT> {
+
+    /**
+     * For internal use only. Set already constructed {@link Window}. This allows easier
+     * construction of composite operators.
+     *
+     * @param window beam window
+     * @return output builder
+     */
+    OutputBuilder<KeyT, OutputT> windowBy(Window<InputT> window);
+  }
+
+  /** Builder for 'windowBy' step */
+  public interface WindowByBuilder<KeyT, OutputT>
+      extends Builders.WindowBy<TriggeredByBuilder<KeyT, OutputT>>,
+          OptionalMethodBuilder<WindowByBuilder<KeyT, OutputT>, OutputBuilder<KeyT, OutputT>>,
+          OutputBuilder<KeyT, OutputT> {
+
+    @Override
+    <W extends BoundedWindow> TriggeredByBuilder<KeyT, OutputT> windowBy(
+        WindowFn<Object, W> windowing);
+
+    @Override
+    default OutputBuilder<KeyT, OutputT> applyIf(
+        boolean cond,
+        UnaryFunction<WindowByBuilder<KeyT, OutputT>, OutputBuilder<KeyT, OutputT>> fn) {
+      requireNonNull(fn);
+      return cond ? fn.apply(this) : this;
+    }
+  }
+
+  /** Builder for 'triggeredBy' step */
+  public interface TriggeredByBuilder<KeyT, OutputT>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<KeyT, OutputT>> {
+
+    @Override
+    AccumulationModeBuilder<KeyT, OutputT> triggeredBy(Trigger trigger);
+  }
+
+  /** Builder for 'accumulationMode' step */
+  public interface AccumulationModeBuilder<KeyT, OutputT>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<KeyT, OutputT>> {
+
+    @Override
+    WindowedOutputBuilder<KeyT, OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode);
+  }
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<KeyT, OutputT>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<KeyT, OutputT>>,
+          OutputBuilder<KeyT, OutputT> {}
+
+  /** Builder for 'output' step */
+  public interface OutputBuilder<KeyT, OutputT>
+      extends Builders.Output<KV<KeyT, OutputT>>, Builders.OutputValues<KeyT, OutputT> {}
+
+  /**
+   * Builder for ReduceByKey operator.
+   *
+   * @param <InputT> type of input
+   * @param <KeyT> type of key
+   * @param <ValueT> type of value
+   * @param <OutputT> type ouf output
+   */
+  static class Builder<InputT, KeyT, ValueT, OutputT>
+      implements OfBuilder,
+          KeyByBuilder<InputT>,
+          ValueByReduceByBuilder<InputT, KeyT, ValueT>,
+          WithSortedValuesBuilder<KeyT, ValueT, OutputT>,
+          WindowByInternalBuilder<InputT, KeyT, OutputT>,
+          WindowByBuilder<KeyT, OutputT>,
+          TriggeredByBuilder<KeyT, OutputT>,
+          AccumulationModeBuilder<KeyT, OutputT>,
+          WindowedOutputBuilder<KeyT, OutputT>,
+          OutputBuilder<KeyT, OutputT> {
+
+    private final WindowBuilder<InputT> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryFunction<InputT, KeyT> keyExtractor;
+    @Nullable private TypeDescriptor<KeyT> keyType;
+    @Nullable private UnaryFunction<InputT, ValueT> valueExtractor;
+    @Nullable private TypeDescriptor<ValueT> valueType;
+    private ReduceFunctor<ValueT, OutputT> reducer;
+    @Nullable private TypeDescriptor<OutputT> outputType;
+    @Nullable private BinaryFunction<ValueT, ValueT, Integer> valueComparator;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> KeyByBuilder<T> of(Dataset<T> input) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, ?, ?, ?> casted = (Builder) this;
+      casted.input = input;
+      return casted;
+    }
+
+    @Override
+    public <T> ValueByReduceByBuilder<InputT, T, InputT> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T, InputT, ?> casted = (Builder) this;
+      casted.keyExtractor = requireNonNull(keyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <T> ReduceByBuilder<KeyT, T> valueBy(
+        UnaryFunction<InputT, T> valueExtractor, @Nullable TypeDescriptor<T> valueType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, KeyT, T, ?> casted = (Builder) this;
+      casted.valueExtractor = requireNonNull(valueExtractor);
+      casted.valueType = valueType;
+      return casted;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> WithSortedValuesBuilder<KeyT, ValueT, T> reduceBy(
+        ReduceFunctor<ValueT, T> reducer, @Nullable TypeDescriptor<T> outputType) {
+      if (valueExtractor == null) {
+        // if the valueExtractor was not set in 'valueBy' step, we use untouched input element
+        valueExtractor = (UnaryFunction) UnaryFunction.identity();
+      }
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, KeyT, ValueT, T> casted = (Builder) this;
+      casted.reducer = requireNonNull(reducer);
+      casted.outputType = outputType;
+      return casted;
+    }
+
+    @Override
+    public WindowByBuilder<KeyT, OutputT> withSortedValues(
+        BinaryFunction<ValueT, ValueT, Integer> valueComparator) {
+      this.valueComparator = requireNonNull(valueComparator);
+      return this;
+    }
+
+    @Override
+    public OutputBuilder<KeyT, OutputT> windowBy(Window<InputT> window) {
+      windowBuilder.setWindow(window);
+      return this;
+    }
+
+    @Override
+    public <W extends BoundedWindow> TriggeredByBuilder<KeyT, OutputT> windowBy(
+        WindowFn<Object, W> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<KeyT, OutputT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withAllowedLateness(Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withTimestampCombiner(
+        TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, OutputT> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    public Dataset<KV<KeyT, OutputT>> output(OutputHint... outputHints) {
+      final ReduceByKey<InputT, KeyT, ValueT, OutputT> rbk =
+          new ReduceByKey<>(
+              name,
+              keyExtractor,
+              keyType,
+              valueExtractor,
+              valueType,
+              reducer,
+              valueComparator,
+              windowBuilder.getWindow().orElse(null),
+              TypeDescriptors.kvs(
+                  TypeAwares.orObjects(Optional.ofNullable(keyType)),
+                  TypeAwares.orObjects(Optional.ofNullable(outputType))));
+      return OperatorTransform.apply(rbk, Collections.singletonList(input));
+    }
+
+    @Override
+    public Dataset<OutputT> outputValues(OutputHint... outputHints) {
+      return MapElements.named(name != null ? name + "::extract-values" : null)
+          .of(output(outputHints))
+          .using(KV::getValue, outputType)
+          .output(outputHints);
+    }
+  }
+
+  private final ReduceFunctor<ValueT, OutputT> reducer;
+  private final UnaryFunction<InputT, ValueT> valueExtractor;
+  @Nullable private final BinaryFunction<ValueT, ValueT, Integer> valueComparator;
+  @Nullable private final TypeDescriptor<ValueT> valueType;
+
+  private ReduceByKey(
+      @Nullable String name,
+      UnaryFunction<InputT, KeyT> keyExtractor,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      UnaryFunction<InputT, ValueT> valueExtractor,
+      @Nullable TypeDescriptor<ValueT> valueType,
+      ReduceFunctor<ValueT, OutputT> reducer,
+      @Nullable BinaryFunction<ValueT, ValueT, Integer> valueComparator,
+      @Nullable Window<InputT> window,
+      TypeDescriptor<KV<KeyT, OutputT>> outputType) {
+    super(name, outputType, keyExtractor, keyType, window);
+    this.reducer = reducer;
+    this.valueExtractor = valueExtractor;
+    this.valueType = valueType;
+    this.valueComparator = valueComparator;
+  }
+
+  public ReduceFunctor<ValueT, OutputT> getReducer() {
+    return reducer;
+  }
+
+  public boolean isCombinable() {
+    return reducer.isCombinable();
+  }
+
+  public UnaryFunction<InputT, ValueT> getValueExtractor() {
+    return valueExtractor;
+  }
+
+  public Optional<BinaryFunction<ValueT, ValueT, Integer>> getValueComparator() {
+    return Optional.ofNullable(valueComparator);
+  }
+
+  @Override
+  public Optional<TypeDescriptor<ValueT>> getValueType() {
+    return Optional.ofNullable(valueType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java
new file mode 100644
index 00000000000..8d34a5939c6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Reduces all elements in a window. The operator corresponds to {@link ReduceByKey} with the same
+ * key for all elements, so the actual key is defined only by window.
+ *
+ * <p>Custom windowing can be set, otherwise values from input operator are used.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code [valueBy] ................} value extractor function (default: identity)
+ *   <li>{@code (combineBy | reduceBy)....} {@link CombinableReduceFunction} or {@link
+ *       ReduceFunction} for combinable or non-combinable function
+ *   <li>{@code [withSortedValues] .......} use comparator for sorting values prior to being passed
+ *       to {@link ReduceFunction} function (applicable only for non-combinable version)
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.CONSTANT_IF_COMBINABLE, repartitions = 1)
+public class ReduceWindow<InputT, ValueT, OutputT> extends ShuffleOperator<InputT, Byte, OutputT>
+    implements TypeAware.Value<ValueT>, CompositeOperator<InputT, OutputT> {
+
+  private static final Byte B_ZERO = (byte) 0;
+
+  /**
+   * Starts building a nameless {@link ReduceWindow} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> ValueByReduceByBuilder<InputT, InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link ReduceWindow} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /** Builder for 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> ValueByReduceByBuilder<InputT, InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for 'reduceBy' step */
+  public interface ReduceByBuilder<ValueT> {
+
+    /**
+     * Define a function that reduces all values related to one key into one result object. The
+     * function is not combinable - i.e. partial results cannot be made up before shuffle. To get
+     * better performance use {@link #combineBy} method.
+     *
+     * @param <OutputT> type of output element
+     * @param reducer function that reduces all values into one output object
+     * @return next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    default <OutputT> WithSortedValuesBuilder<ValueT, OutputT> reduceBy(
+        ReduceFunction<ValueT, OutputT> reducer) {
+      return reduceBy(
+          (Stream<ValueT> in, Collector<OutputT> ctx) -> ctx.collect(reducer.apply(in)));
+    }
+
+    default <OutputT> WithSortedValuesBuilder<ValueT, OutputT> reduceBy(
+        ReduceFunction<ValueT, OutputT> reducer, TypeDescriptor<OutputT> outputType) {
+      return reduceBy(
+          (Stream<ValueT> in, Collector<OutputT> ctx) -> ctx.collect(reducer.apply(in)),
+          outputType);
+    }
+
+    /**
+     * Define a function that reduces all values related to one key into one or more result objects.
+     * The function is not combinable - i.e. partial results cannot be made up before shuffle. To
+     * get better performance use {@link #combineBy} method.
+     *
+     * @param <OutputT> type of output element
+     * @param reducer function that reduces all values into output values
+     * @return next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    default <OutputT> WithSortedValuesBuilder<ValueT, OutputT> reduceBy(
+        ReduceFunctor<ValueT, OutputT> reducer) {
+      return reduceBy(reducer, null);
+    }
+
+    <OutputT> WithSortedValuesBuilder<ValueT, OutputT> reduceBy(
+        ReduceFunctor<ValueT, OutputT> reducer, @Nullable TypeDescriptor<OutputT> outputType);
+
+    /**
+     * Define a function that reduces all values related to one key into one result object. The
+     * function is combinable (associative and commutative) so it can be used to compute partial
+     * results before shuffle.
+     *
+     * @param reducer function that reduces all values into one output object
+     * @return next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    default WindowByBuilder<ValueT> combineBy(CombinableReduceFunction<ValueT> reducer) {
+      return reduceBy(ReduceFunctor.of(reducer));
+    }
+
+    default WindowByBuilder<ValueT> combineBy(
+        CombinableReduceFunction<ValueT> reducer, TypeDescriptor<ValueT> outputType) {
+      return reduceBy(ReduceFunctor.of(reducer), outputType);
+    }
+  }
+
+  /** Builder for 'valueBy' / 'reduceBy' step */
+  public interface ValueByReduceByBuilder<InputT, ValueT> extends ReduceByBuilder<ValueT> {
+
+    /**
+     * Specifies the function to derive a value from the {@link ReduceByKey} operator's input
+     * elements to get reduced by a later supplied reduce function.
+     *
+     * @param <T> the type of the extracted values
+     * @param valueExtractor a user defined function to extract values from the processed input
+     *     dataset's elements for later reduction
+     * @param valueType type of extracted value
+     * @return the next builder to complete the setup of the {@link ReduceByKey} operator
+     */
+    <T> ReduceByBuilder<T> valueBy(
+        UnaryFunction<InputT, T> valueExtractor, @Nullable TypeDescriptor<T> valueType);
+
+    default <T> ReduceByBuilder<T> valueBy(UnaryFunction<InputT, T> valueExtractor) {
+      return valueBy(valueExtractor, null);
+    }
+  }
+
+  /** Builder for 'withSortedValues' step */
+  public interface WithSortedValuesBuilder<ValueT, OutputT> extends WindowByBuilder<OutputT> {
+
+    /**
+     * Sort values going to `reduceBy` function by given comparator.
+     *
+     * @param comparator function with contract defined by {@code java.util.Comparator#compare}.
+     * @return next step builder
+     */
+    WindowByBuilder<OutputT> withSortedValues(BinaryFunction<ValueT, ValueT, Integer> comparator);
+  }
+
+  /** Builder for 'windowBy' step */
+  public interface WindowByBuilder<OutputT>
+      extends Builders.WindowBy<TriggeredByBuilder<OutputT>>,
+          OptionalMethodBuilder<WindowByBuilder<OutputT>, OutputBuilder<OutputT>>,
+          OutputBuilder<OutputT> {
+
+    @Override
+    <W extends BoundedWindow> TriggeredByBuilder<OutputT> windowBy(WindowFn<Object, W> windowing);
+
+    @Override
+    default OutputBuilder<OutputT> applyIf(
+        boolean cond, UnaryFunction<WindowByBuilder<OutputT>, OutputBuilder<OutputT>> fn) {
+      requireNonNull(fn);
+      return cond ? fn.apply(this) : this;
+    }
+  }
+
+  /** Builder for 'triggeredBy' step */
+  public interface TriggeredByBuilder<OutputT>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<OutputT>> {
+
+    @Override
+    AccumulationModeBuilder<OutputT> triggeredBy(Trigger trigger);
+  }
+
+  /** Builder for 'accumulationMode' step */
+  public interface AccumulationModeBuilder<OutputT>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<OutputT>> {
+
+    @Override
+    WindowedOutputBuilder<OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode);
+  }
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<OutputT>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<OutputT>>, OutputBuilder<OutputT> {}
+
+  public interface OutputBuilder<OutputT> extends Builders.Output<OutputT> {}
+
+  /**
+   * Builder for ReduceByKey operator.
+   *
+   * @param <InputT> type of input
+   * @param <ValueT> type of value
+   * @param <OutputT> type ouf output
+   */
+  private static class Builder<InputT, ValueT, OutputT>
+      implements OfBuilder,
+          ValueByReduceByBuilder<InputT, ValueT>,
+          WithSortedValuesBuilder<ValueT, OutputT>,
+          WindowByBuilder<OutputT>,
+          TriggeredByBuilder<OutputT>,
+          AccumulationModeBuilder<OutputT>,
+          WindowedOutputBuilder<OutputT>,
+          OutputBuilder<OutputT> {
+
+    private final WindowBuilder<InputT> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    @Nullable private UnaryFunction<InputT, ValueT> valueExtractor;
+    @Nullable private TypeDescriptor<ValueT> valueType;
+    private ReduceFunctor<ValueT, OutputT> reducer;
+    @Nullable private TypeDescriptor<OutputT> outputType;
+    @Nullable private BinaryFunction<ValueT, ValueT, Integer> valueComparator;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T> ValueByReduceByBuilder<T, T> of(Dataset<T> input) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, T, ?> casted = (Builder) this;
+      casted.input = requireNonNull(input);
+      return casted;
+    }
+
+    @Override
+    public <T> ReduceByBuilder<T> valueBy(
+        UnaryFunction<InputT, T> valueExtractor, @Nullable TypeDescriptor<T> valueType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T, ?> casted = (Builder) this;
+      casted.valueExtractor = requireNonNull(valueExtractor);
+      casted.valueType = valueType;
+      return casted;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> WithSortedValuesBuilder<ValueT, T> reduceBy(
+        ReduceFunctor<ValueT, T> reducer, @Nullable TypeDescriptor<T> outputType) {
+      if (valueExtractor == null) {
+        // if the valueExtractor was not set in 'valueBy' step, we use untouched input element
+        valueExtractor = (UnaryFunction) UnaryFunction.identity();
+      }
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, ValueT, T> casted = (Builder) this;
+      casted.reducer = requireNonNull(reducer);
+      casted.outputType = outputType;
+      return casted;
+    }
+
+    @Override
+    public WindowByBuilder<OutputT> withSortedValues(
+        BinaryFunction<ValueT, ValueT, Integer> valueComparator) {
+      this.valueComparator = requireNonNull(valueComparator);
+      return this;
+    }
+
+    @Override
+    public <T extends BoundedWindow> TriggeredByBuilder<OutputT> windowBy(
+        WindowFn<Object, T> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<OutputT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withAllowedLateness(Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withTimestampCombiner(
+        TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<OutputT> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    public Dataset<OutputT> output(OutputHint... outputHints) {
+      final ReduceWindow<InputT, ValueT, OutputT> rw =
+          new ReduceWindow<>(
+              name,
+              valueExtractor,
+              valueType,
+              reducer,
+              valueComparator,
+              windowBuilder.getWindow().orElse(null),
+              outputType);
+      return OperatorTransform.apply(rw, Collections.singletonList(input));
+    }
+  }
+
+  private final ReduceFunctor<ValueT, OutputT> reducer;
+  private final UnaryFunction<InputT, ValueT> valueExtractor;
+  @Nullable private final BinaryFunction<ValueT, ValueT, Integer> valueComparator;
+  @Nullable private final TypeDescriptor<ValueT> valueType;
+
+  private ReduceWindow(
+      @Nullable String name,
+      UnaryFunction<InputT, ValueT> valueExtractor,
+      @Nullable TypeDescriptor<ValueT> valueType,
+      ReduceFunctor<ValueT, OutputT> reducer,
+      @Nullable BinaryFunction<ValueT, ValueT, Integer> valueComparator,
+      @Nullable Window<InputT> window,
+      TypeDescriptor<OutputT> outputType) {
+
+    super(name, outputType, e -> B_ZERO, TypeDescriptors.bytes(), window);
+    this.reducer = reducer;
+    this.valueExtractor = valueExtractor;
+    this.valueType = valueType;
+    this.valueComparator = valueComparator;
+  }
+
+  public ReduceFunctor<ValueT, OutputT> getReducer() {
+    return reducer;
+  }
+
+  public boolean isCombinable() {
+    return reducer.isCombinable();
+  }
+
+  public UnaryFunction<InputT, ValueT> getValueExtractor() {
+    return valueExtractor;
+  }
+
+  public Optional<BinaryFunction<ValueT, ValueT, Integer>> getValueComparator() {
+    return Optional.ofNullable(valueComparator);
+  }
+
+  @Override
+  public Optional<TypeDescriptor<ValueT>> getValueType() {
+    return Optional.ofNullable(valueType);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Dataset<OutputT> expand(List<Dataset<InputT>> inputs) {
+    final ReduceByKey.ReduceByBuilder<Byte, ValueT> reduceBy =
+        ReduceByKey.named(getName().orElse(null))
+            .of(Iterables.getOnlyElement(inputs))
+            .keyBy(e -> B_ZERO)
+            .valueBy(valueExtractor, valueType);
+    final ReduceByKey.WithSortedValuesBuilder<Byte, ValueT, OutputT> sortBy =
+        reduceBy.reduceBy(reducer);
+    if (isCombinable()) {
+      // sanity check
+      checkState(valueComparator == null, "Sorting is not supported for combinable reducers.");
+    }
+    final ReduceByKey.WindowByBuilder<Byte, OutputT> windowBy =
+        getValueComparator().isPresent()
+            ? sortBy.withSortedValues(getValueComparator().get())
+            : sortBy;
+    return windowBy
+        .applyIf(
+            getWindow().isPresent(),
+            builder -> {
+              @SuppressWarnings("unchecked")
+              final ReduceByKey.WindowByInternalBuilder<InputT, Byte, OutputT> casted =
+                  (ReduceByKey.WindowByInternalBuilder) builder;
+              return casted.windowBy(
+                  getWindow()
+                      .orElseThrow(
+                          () ->
+                              new IllegalStateException(
+                                  "Unable to resolve windowing for ReduceWindow expansion.")));
+            })
+        .outputValues();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java
new file mode 100644
index 00000000000..65bb4cabd65
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join.Type;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Right outer join of two input datasets producing single new dataset.
+ *
+ * <p>When joining two streams, the join has to specify windowing which groups elements from streams
+ * into {@link org.apache.beam.sdk.transforms.windowing.Window}s. The join operation is performed
+ * within same windows produced on left and right side of input {@link Dataset}s.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} left and right input dataset
+ *   <li>{@code by .......................} {@link UnaryFunction}s transforming left and right
+ *       elements into keys
+ *   <li>{@code using ....................} {@link BinaryFunctor} receiving left and right element
+ *       from joined window
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code (output | outputValues) ..} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+public class RightJoin {
+
+  /**
+   * Create builder.
+   *
+   * @param left dataset
+   * @param right dataset
+   * @param <LeftT> type of left dataset
+   * @param <RightT> type of right dataset
+   * @return ByBuilder
+   */
+  public static <LeftT, RightT> ByBuilder<LeftT, RightT> of(
+      Dataset<LeftT> left, Dataset<RightT> right) {
+    return named("RightJoin").of(left, right);
+  }
+
+  /**
+   * Optional setter to give operator name.
+   *
+   * @param name of the operator
+   * @return OfBuilder
+   */
+  public static OfBuilder named(String name) {
+    return new Builder<>(name);
+  }
+
+  /** Builder for the 'of' step */
+  public interface OfBuilder {
+
+    <LeftT, RightT> ByBuilder<LeftT, RightT> of(Dataset<LeftT> left, Dataset<RightT> right);
+  }
+
+  /** Builder for the 'by' step */
+  public interface ByBuilder<LeftT, RightT> {
+
+    <KeyT> UsingBuilder<LeftT, RightT, KeyT> by(
+        UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+        UnaryFunction<RightT, KeyT> rightKeyExtractor,
+        @Nullable TypeDescriptor<KeyT> keyType);
+
+    default <KeyT> UsingBuilder<LeftT, RightT, KeyT> by(
+        UnaryFunction<LeftT, KeyT> leftKeyExtractor,
+        UnaryFunction<RightT, KeyT> rightKeyExtractor) {
+      return by(leftKeyExtractor, rightKeyExtractor, null);
+    }
+  }
+
+  /** Builder for the 'using' step */
+  public interface UsingBuilder<LeftT, RightT, KeyT> {
+
+    <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<Optional<LeftT>, RightT, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputType);
+
+    default <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<Optional<LeftT>, RightT, OutputT> joinFunc) {
+      return using(joinFunc, null);
+    }
+  }
+
+  private static class Builder<LeftT, RightT, KeyT>
+      implements OfBuilder, ByBuilder<LeftT, RightT>, UsingBuilder<LeftT, RightT, KeyT> {
+
+    private final String name;
+    private Dataset<LeftT> left;
+    private Dataset<RightT> right;
+    private UnaryFunction<LeftT, KeyT> leftKeyExtractor;
+    private UnaryFunction<RightT, KeyT> rightKeyExtractor;
+    @Nullable TypeDescriptor<KeyT> keyType;
+
+    private Builder(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T, S> ByBuilder<T, S> of(Dataset<T> left, Dataset<S> right) {
+      @SuppressWarnings("unchecked")
+      final Builder<T, S, ?> casted = (Builder) this;
+      casted.left = requireNonNull(left);
+      casted.right = requireNonNull(right);
+      return casted;
+    }
+
+    @Override
+    public <T> UsingBuilder<LeftT, RightT, T> by(
+        UnaryFunction<LeftT, T> leftKeyExtractor,
+        UnaryFunction<RightT, T> rightKeyExtractor,
+        @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<LeftT, RightT, T> casted = (Builder) this;
+      casted.leftKeyExtractor = requireNonNull(leftKeyExtractor);
+      casted.rightKeyExtractor = requireNonNull(rightKeyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <OutputT> Join.WindowByBuilder<KeyT, OutputT> using(
+        BinaryFunctor<Optional<LeftT>, RightT, OutputT> joinFunc,
+        @Nullable TypeDescriptor<OutputT> outputType) {
+      return new Join.Builder<>(name, Type.RIGHT)
+          .of(left, right)
+          .by(leftKeyExtractor, rightKeyExtractor, keyType)
+          .using(
+              (LeftT l, RightT r, Collector<OutputT> c) ->
+                  joinFunc.apply(Optional.ofNullable(l), r, c),
+              outputType);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java
new file mode 100644
index 00000000000..2eba430e8f5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeUtils;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Operator for summing of long values extracted from elements. The sum is operated upon defined key
+ * and window.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * Dataset<KV<String, Long>> summed = SumByKey.of(elements)
+ *     .keyBy(KV::getKey)
+ *     .valueBy(KV::getValue)
+ *     .output();
+ * }</pre>
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code keyBy ....................} key extractor function
+ *   <li>{@code [valueBy] ................} {@link UnaryFunction} transforming from input element to
+ *       long (default: {@code e -> 1L})
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code (output | outputValues) ..} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.CONSTANT, repartitions = 1)
+public class SumByKey<InputT, KeyT> extends ShuffleOperator<InputT, KeyT, KV<KeyT, Long>>
+    implements CompositeOperator<InputT, KV<KeyT, Long>> {
+
+  /**
+   * Starts building a nameless {@link SumByKey} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link SumByKey} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder(name);
+  }
+
+  /** Builder for 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for 'keyBy' step */
+  public interface KeyByBuilder<InputT> extends Builders.KeyBy<InputT> {
+
+    @Override
+    <T> ValueByBuilder<InputT, T> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, TypeDescriptor<T> keyType);
+
+    @Override
+    default <T> ValueByBuilder<InputT, T> keyBy(UnaryFunction<InputT, T> keyExtractor) {
+      return keyBy(keyExtractor, null);
+    }
+  }
+
+  /** Builder for 'valueBy' step */
+  public interface ValueByBuilder<InputT, KeyT> extends WindowByBuilder<KeyT> {
+
+    WindowByBuilder<KeyT> valueBy(UnaryFunction<InputT, Long> valueExtractor);
+  }
+
+  /** Builder for 'windowBy' step */
+  public interface WindowByBuilder<KeyT>
+      extends Builders.WindowBy<TriggeredByBuilder<KeyT>>,
+          OptionalMethodBuilder<WindowByBuilder<KeyT>, OutputBuilder<KeyT>>,
+          OutputBuilder<KeyT> {
+
+    @Override
+    <W extends BoundedWindow> TriggeredByBuilder<KeyT> windowBy(WindowFn<Object, W> windowing);
+
+    @Override
+    default OutputBuilder<KeyT> applyIf(
+        boolean cond, UnaryFunction<WindowByBuilder<KeyT>, OutputBuilder<KeyT>> fn) {
+      return cond ? requireNonNull(fn).apply(this) : this;
+    }
+  }
+
+  /** Builder for 'triggeredBy' step */
+  public interface TriggeredByBuilder<KeyT>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<KeyT>> {
+
+    @Override
+    AccumulationModeBuilder<KeyT> triggeredBy(Trigger trigger);
+  }
+
+  /** Builder for 'accumulationMode' step */
+  public interface AccumulationModeBuilder<KeyT>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<KeyT>> {
+
+    @Override
+    WindowedOutputBuilder<KeyT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode);
+  }
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<KeyT>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<KeyT>>, OutputBuilder<KeyT> {}
+
+  /** Builder for 'output' step */
+  public interface OutputBuilder<KeyT> extends Builders.Output<KV<KeyT, Long>> {}
+
+  /**
+   * Builder for SumByKey operator.
+   *
+   * @param <InputT> type of input
+   * @param <KeyT> type of key
+   */
+  private static class Builder<InputT, KeyT>
+      implements OfBuilder,
+          KeyByBuilder<InputT>,
+          ValueByBuilder<InputT, KeyT>,
+          WindowByBuilder<KeyT>,
+          TriggeredByBuilder<KeyT>,
+          AccumulationModeBuilder<KeyT>,
+          WindowedOutputBuilder<KeyT>,
+          OutputBuilder<KeyT> {
+
+    private final WindowBuilder<InputT> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryFunction<InputT, KeyT> keyExtractor;
+    @Nullable private TypeDescriptor<KeyT> keyType;
+    private UnaryFunction<InputT, Long> valueExtractor;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> KeyByBuilder<T> of(Dataset<T> input) {
+      this.input = (Dataset<InputT>) requireNonNull(input);
+      return (KeyByBuilder) this;
+    }
+
+    @Override
+    public <T> ValueByBuilder<InputT, T> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T> casted = (Builder<InputT, T>) this;
+      casted.keyExtractor = requireNonNull(keyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public WindowByBuilder<KeyT> valueBy(UnaryFunction<InputT, Long> valueExtractor) {
+      this.valueExtractor = requireNonNull(valueExtractor);
+      return this;
+    }
+
+    @Override
+    public <W extends BoundedWindow> TriggeredByBuilder<KeyT> windowBy(
+        WindowFn<Object, W> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<KeyT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withAllowedLateness(Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withTimestampCombiner(TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    public Dataset<KV<KeyT, Long>> output(OutputHint... outputHints) {
+      if (valueExtractor == null) {
+        valueExtractor = e -> 1L;
+      }
+      final SumByKey<InputT, KeyT> sbk =
+          new SumByKey<>(
+              name,
+              keyExtractor,
+              keyType,
+              valueExtractor,
+              windowBuilder.getWindow().orElse(null),
+              TypeUtils.keyValues(
+                  TypeAwares.orObjects(Optional.ofNullable(keyType)), TypeDescriptors.longs()));
+      return OperatorTransform.apply(sbk, Collections.singletonList(input));
+    }
+  }
+
+  private final UnaryFunction<InputT, Long> valueExtractor;
+
+  private SumByKey(
+      @Nullable String name,
+      UnaryFunction<InputT, KeyT> keyExtractor,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      UnaryFunction<InputT, Long> valueExtractor,
+      @Nullable Window<InputT> window,
+      TypeDescriptor<KV<KeyT, Long>> outputType) {
+    super(name, outputType, keyExtractor, keyType, window);
+    this.valueExtractor = valueExtractor;
+  }
+
+  public UnaryFunction<InputT, Long> getValueExtractor() {
+    return valueExtractor;
+  }
+
+  @Override
+  public Dataset<KV<KeyT, Long>> expand(List<Dataset<InputT>> inputs) {
+    return ReduceByKey.named(getName().orElse(null))
+        .of(Iterables.getOnlyElement(inputs))
+        .keyBy(getKeyExtractor())
+        .valueBy(getValueExtractor(), TypeDescriptors.longs())
+        .combineBy(Sums.ofLongs())
+        .applyIf(
+            getWindow().isPresent(),
+            builder -> {
+              @SuppressWarnings("unchecked")
+              final ReduceByKey.WindowByInternalBuilder<InputT, KeyT, Long> casted =
+                  (ReduceByKey.WindowByInternalBuilder) builder;
+              return casted.windowBy(
+                  getWindow()
+                      .orElseThrow(
+                          () ->
+                              new IllegalStateException(
+                                  "Unable to resolve windowing for SumByKey expansion.")));
+            })
+        .output();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java
new file mode 100644
index 00000000000..86abfdc416b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.collect.Iterables;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.OptionalMethodBuilder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeUtils;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Emits top element for defined keys and windows. The elements are compared by comparable objects
+ * extracted by user defined function applied on input elements.
+ *
+ * <p>Custom windowing can be set, otherwise values from input operator are used.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * TopPerKey.of(elements)
+ *      .keyBy(e -> (byte) 0)
+ *      .valueBy(e -> e)
+ *      .scoreBy(KV::getValue)
+ *      .output();
+ * }</pre>
+ *
+ * <p>The examples above finds global maximum of all elements.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input dataset
+ *   <li>{@code keyBy ....................} key extractor function
+ *   <li>{@code valueBy ..................} value extractor function
+ *   <li>{@code scoreBy ..................} {@link UnaryFunction} transforming input elements to
+ *       {@link Comparable} scores
+ *   <li>{@code [windowBy] ...............} windowing (see {@link WindowFn}), default is no
+ *       windowing
+ *   <li>{@code [triggeredBy] ............} defines windowing trigger, follows [windowBy] if called
+ *   <li>{@code [accumulationMode] .......} windowing accumulation mode, follows [triggeredBy]
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Derived(state = StateComplexity.CONSTANT, repartitions = 1)
+public class TopPerKey<InputT, KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+    extends ShuffleOperator<InputT, KeyT, Triple<KeyT, ValueT, ScoreT>>
+    implements TypeAware.Value<ValueT>, CompositeOperator<InputT, Triple<KeyT, ValueT, ScoreT>> {
+
+  /**
+   * Starts building a nameless {@link TopPerKey} operator to process the given input dataset.
+   *
+   * @param <InputT> the type of elements of the input dataset
+   * @param input the input data set to be processed
+   * @return a builder to complete the setup of the new operator
+   * @see #named(String)
+   * @see OfBuilder#of(Dataset)
+   */
+  public static <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input) {
+    return named(null).of(input);
+  }
+
+  /**
+   * Starts building a named {@link TopPerKey} operator.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return a builder to complete the setup of the new operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder<>(name);
+  }
+
+  /** Builder for 'of' step */
+  public interface OfBuilder extends Builders.Of {
+
+    @Override
+    <InputT> KeyByBuilder<InputT> of(Dataset<InputT> input);
+  }
+
+  /** Builder for 'keyBy' step */
+  public interface KeyByBuilder<InputT> extends Builders.KeyBy<InputT> {
+
+    @Override
+    <T> ValueByBuilder<InputT, T> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, TypeDescriptor<T> keyType);
+
+    @Override
+    default <T> ValueByBuilder<InputT, T> keyBy(UnaryFunction<InputT, T> keyExtractor) {
+      return keyBy(keyExtractor, null);
+    }
+  }
+
+  /** Builder for 'valueBy' step */
+  public interface ValueByBuilder<InputT, KeyT> {
+
+    default <ValueT> ScoreBy<InputT, KeyT, ValueT> valueBy(
+        UnaryFunction<InputT, ValueT> valueExtractor) {
+      return valueBy(valueExtractor, null);
+    }
+
+    <ValueT> ScoreBy<InputT, KeyT, ValueT> valueBy(
+        UnaryFunction<InputT, ValueT> valueExtractor, @Nullable TypeDescriptor<ValueT> valueType);
+  }
+
+  /** Builder for 'scoreBy' step */
+  public interface ScoreBy<InputT, KeyT, ValueT> {
+
+    default <ScoreT extends Comparable<ScoreT>> WindowByBuilder<KeyT, ValueT, ScoreT> scoreBy(
+        UnaryFunction<InputT, ScoreT> scoreFn) {
+      return scoreBy(scoreFn, null);
+    }
+
+    <ScoreT extends Comparable<ScoreT>> WindowByBuilder<KeyT, ValueT, ScoreT> scoreBy(
+        UnaryFunction<InputT, ScoreT> scoreFn, @Nullable TypeDescriptor<ScoreT> scoreType);
+  }
+
+  /** Builder for 'windowBy' step */
+  public interface WindowByBuilder<KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+      extends Builders.WindowBy<TriggeredByBuilder<KeyT, ValueT, ScoreT>>,
+          OptionalMethodBuilder<
+              WindowByBuilder<KeyT, ValueT, ScoreT>, OutputBuilder<KeyT, ValueT, ScoreT>>,
+          OutputBuilder<KeyT, ValueT, ScoreT> {
+
+    @Override
+    <W extends BoundedWindow> TriggeredByBuilder<KeyT, ValueT, ScoreT> windowBy(
+        WindowFn<Object, W> windowing);
+
+    @Override
+    default OutputBuilder<KeyT, ValueT, ScoreT> applyIf(
+        boolean cond,
+        UnaryFunction<WindowByBuilder<KeyT, ValueT, ScoreT>, OutputBuilder<KeyT, ValueT, ScoreT>>
+            fn) {
+      return cond ? requireNonNull(fn).apply(this) : this;
+    }
+  }
+
+  /** Builder for 'triggeredBy' step */
+  public interface TriggeredByBuilder<KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+      extends Builders.TriggeredBy<AccumulationModeBuilder<KeyT, ValueT, ScoreT>> {
+
+    @Override
+    AccumulationModeBuilder<KeyT, ValueT, ScoreT> triggeredBy(Trigger trigger);
+  }
+
+  /** Builder for 'accumulationMode' step */
+  public interface AccumulationModeBuilder<KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+      extends Builders.AccumulationMode<WindowedOutputBuilder<KeyT, ValueT, ScoreT>> {
+
+    @Override
+    WindowedOutputBuilder<KeyT, ValueT, ScoreT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode);
+  }
+
+  /** Builder for 'windowed output' step */
+  public interface WindowedOutputBuilder<KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+      extends Builders.WindowedOutput<WindowedOutputBuilder<KeyT, ValueT, ScoreT>>,
+          OutputBuilder<KeyT, ValueT, ScoreT> {}
+
+  /** Builder for 'output' step */
+  public interface OutputBuilder<KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+      extends Builders.Output<Triple<KeyT, ValueT, ScoreT>> {}
+
+  /**
+   * Builder for TopPerKey operator.
+   *
+   * @param <InputT> type of input
+   * @param <KeyT> type of key
+   */
+  private static class Builder<InputT, KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
+      implements OfBuilder,
+          KeyByBuilder<InputT>,
+          ValueByBuilder<InputT, KeyT>,
+          ScoreBy<InputT, KeyT, ValueT>,
+          WindowByBuilder<KeyT, ValueT, ScoreT>,
+          TriggeredByBuilder<KeyT, ValueT, ScoreT>,
+          AccumulationModeBuilder<KeyT, ValueT, ScoreT>,
+          WindowedOutputBuilder<KeyT, ValueT, ScoreT>,
+          OutputBuilder<KeyT, ValueT, ScoreT> {
+
+    private final WindowBuilder<InputT> windowBuilder = new WindowBuilder<>();
+
+    @Nullable private final String name;
+    private Dataset<InputT> input;
+    private UnaryFunction<InputT, KeyT> keyExtractor;
+    @Nullable private TypeDescriptor<KeyT> keyType;
+    private UnaryFunction<InputT, ValueT> valueExtractor;
+    @Nullable private TypeDescriptor<ValueT> valueType;
+    private UnaryFunction<InputT, ScoreT> scoreExtractor;
+    @Nullable private TypeDescriptor<ScoreT> scoreType;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> KeyByBuilder<T> of(Dataset<T> input) {
+      this.input = (Dataset<InputT>) requireNonNull(input);
+      return (KeyByBuilder) this;
+    }
+
+    @Override
+    public <T> ValueByBuilder<InputT, T> keyBy(
+        UnaryFunction<InputT, T> keyExtractor, @Nullable TypeDescriptor<T> keyType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, T, ?, ?> casted = (Builder) this;
+      casted.keyExtractor = requireNonNull(keyExtractor);
+      casted.keyType = keyType;
+      return casted;
+    }
+
+    @Override
+    public <T> ScoreBy<InputT, KeyT, T> valueBy(
+        UnaryFunction<InputT, T> valueExtractor, @Nullable TypeDescriptor<T> valueType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, KeyT, T, ?> casted = (Builder) this;
+      casted.valueExtractor = requireNonNull(valueExtractor);
+      casted.valueType = valueType;
+      return casted;
+    }
+
+    @Override
+    public <T extends Comparable<T>> WindowByBuilder<KeyT, ValueT, T> scoreBy(
+        UnaryFunction<InputT, T> scoreExtractor, @Nullable TypeDescriptor<T> scoreType) {
+      @SuppressWarnings("unchecked")
+      final Builder<InputT, KeyT, ValueT, T> casted = (Builder) this;
+      casted.scoreExtractor = requireNonNull(scoreExtractor);
+      casted.scoreType = scoreType;
+      return casted;
+    }
+
+    @Override
+    public <W extends BoundedWindow> TriggeredByBuilder<KeyT, ValueT, ScoreT> windowBy(
+        WindowFn<Object, W> windowFn) {
+      windowBuilder.windowBy(windowFn);
+      return this;
+    }
+
+    @Override
+    public AccumulationModeBuilder<KeyT, ValueT, ScoreT> triggeredBy(Trigger trigger) {
+      windowBuilder.triggeredBy(trigger);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, ValueT, ScoreT> accumulationMode(
+        WindowingStrategy.AccumulationMode accumulationMode) {
+      windowBuilder.accumulationMode(accumulationMode);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, ValueT, ScoreT> withAllowedLateness(
+        Duration allowedLateness) {
+      windowBuilder.withAllowedLateness(allowedLateness);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, ValueT, ScoreT> withAllowedLateness(
+        Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+      windowBuilder.withAllowedLateness(allowedLateness, closingBehavior);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, ValueT, ScoreT> withTimestampCombiner(
+        TimestampCombiner timestampCombiner) {
+      windowBuilder.withTimestampCombiner(timestampCombiner);
+      return this;
+    }
+
+    @Override
+    public WindowedOutputBuilder<KeyT, ValueT, ScoreT> withOnTimeBehavior(
+        Window.OnTimeBehavior behavior) {
+      windowBuilder.withOnTimeBehavior(behavior);
+      return this;
+    }
+
+    @Override
+    public Dataset<Triple<KeyT, ValueT, ScoreT>> output(OutputHint... outputHints) {
+      final TopPerKey<InputT, KeyT, ValueT, ScoreT> sbk =
+          new TopPerKey<>(
+              name,
+              keyExtractor,
+              keyType,
+              valueExtractor,
+              valueType,
+              scoreExtractor,
+              scoreType,
+              windowBuilder.getWindow().orElse(null),
+              TypeUtils.triplets(keyType, valueType, scoreType));
+      return OperatorTransform.apply(sbk, Collections.singletonList(input));
+    }
+  }
+
+  private UnaryFunction<InputT, ValueT> valueExtractor;
+  @Nullable private TypeDescriptor<ValueT> valueType;
+  private UnaryFunction<InputT, ScoreT> scoreExtractor;
+  @Nullable private TypeDescriptor<ScoreT> scoreType;
+
+  private TopPerKey(
+      @Nullable String name,
+      UnaryFunction<InputT, KeyT> keyExtractor,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      UnaryFunction<InputT, ValueT> valueExtractor,
+      @Nullable TypeDescriptor<ValueT> valueType,
+      UnaryFunction<InputT, ScoreT> scoreExtractor,
+      @Nullable TypeDescriptor<ScoreT> scoreType,
+      @Nullable Window<InputT> window,
+      @Nullable TypeDescriptor<Triple<KeyT, ValueT, ScoreT>> outputType) {
+    super(name, outputType, keyExtractor, keyType, window);
+
+    this.valueExtractor = valueExtractor;
+    this.valueType = valueType;
+    this.scoreExtractor = scoreExtractor;
+    this.scoreType = scoreType;
+  }
+
+  public UnaryFunction<InputT, ValueT> getValueExtractor() {
+    return valueExtractor;
+  }
+
+  @Override
+  public Optional<TypeDescriptor<ValueT>> getValueType() {
+    return Optional.ofNullable(valueType);
+  }
+
+  public UnaryFunction<InputT, ScoreT> getScoreExtractor() {
+    return scoreExtractor;
+  }
+
+  public Optional<TypeDescriptor<ScoreT>> getScoreType() {
+    return Optional.ofNullable(scoreType);
+  }
+
+  @Override
+  public Dataset<Triple<KeyT, ValueT, ScoreT>> expand(List<Dataset<InputT>> inputs) {
+    final Dataset<Triple<KeyT, ValueT, ScoreT>> extracted =
+        MapElements.named("extract-key-value-score")
+            .of(Iterables.getOnlyElement(inputs))
+            .using(
+                elem ->
+                    Triple.of(
+                        getKeyExtractor().apply(elem),
+                        getValueExtractor().apply(elem),
+                        getScoreExtractor().apply(elem)),
+                getOutputType().orElse(null))
+            .output();
+    return ReduceByKey.named("combine-by-key")
+        .of(extracted)
+        .keyBy(Triple::getFirst, getKeyType().orElse(null))
+        .combineBy(
+            (Stream<Triple<KeyT, ValueT, ScoreT>> triplets) ->
+                triplets
+                    .reduce((a, b) -> a.getThird().compareTo(b.getThird()) > 0 ? a : b)
+                    .orElseThrow(IllegalStateException::new))
+        .applyIf(
+            getWindow().isPresent(),
+            builder -> {
+              @SuppressWarnings("unchecked")
+              final ReduceByKey.WindowByInternalBuilder<InputT, KeyT, Triple<KeyT, ValueT, ScoreT>>
+                  casted = (ReduceByKey.WindowByInternalBuilder) builder;
+              return casted.windowBy(
+                  getWindow()
+                      .orElseThrow(
+                          () ->
+                              new IllegalStateException(
+                                  "Unable to resolve windowing for TopPerKey expansion.")));
+            })
+        .outputValues();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java
new file mode 100644
index 00000000000..5ae7b40dd9e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Arrays;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTransform;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * The union of at least two datasets of the same type.
+ *
+ * <p>In the context of Euphoria, a union is merely a logical view on two datasets as one. One can
+ * think of a union as a plain concatenation of at least two dataset without any guarantees about
+ * the order of the datasets' elements. Unlike in set theory, such a union has no notion of
+ * uniqueness, i.e. if the two input dataset contain the same elements, these will all appear in the
+ * output dataset (as duplicates) untouched.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * Dataset<String> xs = ...;
+ * Dataset<String> ys = ...;
+ * Dataset<String> both = Union.named("XS-AND-YS").of(xs, ys).output();
+ * }</pre>
+ *
+ * <p>The "both" dataset from the above example can now be processed with an operator expecting only
+ * a single input dataset, e.g. {@link FlatMap}, which will then effectively process both "xs" and
+ * "ys".
+ *
+ * <p>Note: the order of the dataset does not matter. Indeed, the order of the elements themselves
+ * in the union is intentionally not specified at all.
+ *
+ * <h3>Builders:</h3>
+ *
+ * <ol>
+ *   <li>{@code [named] ..................} give name to the operator [optional]
+ *   <li>{@code of .......................} input datasets
+ *   <li>{@code output ...................} build output dataset
+ * </ol>
+ */
+@Audience(Audience.Type.CLIENT)
+@Basic(state = StateComplexity.ZERO, repartitions = 0)
+public class Union<InputT> extends Operator<InputT> {
+
+  /**
+   * Starts building a nameless Union operator to view at least two datasets as one.
+   *
+   * @param <InputT> the type of elements in the data sets
+   * @param dataSets at least the two data sets
+   * @return the next builder to complete the setup of the {@link Union} operator
+   * @see #named(String)
+   * @see OfBuilder#of(List)
+   */
+  @SafeVarargs
+  public static <InputT> OutputBuilder<InputT> of(Dataset<InputT>... dataSets) {
+    return of(Arrays.asList(dataSets));
+  }
+
+  /**
+   * Starts building a nameless Union operator to view at least two datasets as one.
+   *
+   * @param <InputT> the type of elements in the data sets
+   * @param dataSets at least the two data sets
+   * @return the next builder to complete the setup of the {@link Union} operator
+   * @see #named(String)
+   * @see OfBuilder#of(List)
+   */
+  public static <InputT> OutputBuilder<InputT> of(List<Dataset<InputT>> dataSets) {
+    return named(null).of(dataSets);
+  }
+
+  /**
+   * Starts building a named {@link Union} operator to view two datasets as one.
+   *
+   * @param name a user provided name of the new operator to build
+   * @return the next builder to complete the setup of the new {@link Union} operator
+   */
+  public static OfBuilder named(@Nullable String name) {
+    return new Builder<>(name);
+  }
+
+  /** Builder for the 'of' step */
+  public abstract static class OfBuilder {
+
+    /**
+     * Specifies the two data sets to be "unioned".
+     *
+     * @param <InputT> the type of elements in the two datasets
+     * @param dataSets at least two datSets
+     * @return the next builder to complete the setup of the {@link Union} operator
+     */
+    @SafeVarargs
+    public final <InputT> OutputBuilder<InputT> of(Dataset<InputT>... dataSets) {
+      return of(Arrays.asList(dataSets));
+    }
+
+    /**
+     * Specifies the two data sets to be "unioned".
+     *
+     * @param <InputT> the type of elements in the two datasets
+     * @param dataSets at least two datSets
+     * @return the next builder to complete the setup of the {@link Union} operator
+     */
+    public abstract <InputT> OutputBuilder<InputT> of(List<Dataset<InputT>> dataSets);
+  }
+
+  /**
+   * Last builder in a chain. It concludes this operators creation by calling {@link
+   * #output(OutputHint...)}.
+   */
+  public interface OutputBuilder<InputT> extends Builders.Output<InputT> {}
+
+  private static class Builder<InputT> extends OfBuilder implements OutputBuilder<InputT> {
+
+    @Nullable private final String name;
+    private List<Dataset<InputT>> dataSets;
+
+    Builder(@Nullable String name) {
+      this.name = name;
+    }
+
+    @Override
+    public <T> OutputBuilder<T> of(List<Dataset<T>> dataSets) {
+      @SuppressWarnings("unchecked")
+      final Builder<T> casted = (Builder) this;
+      casted.dataSets = dataSets;
+      return casted;
+    }
+
+    @Override
+    public Dataset<InputT> output(OutputHint... outputHints) {
+      checkArgument(dataSets.size() > 1, "Union needs at least two data sets.");
+      return OperatorTransform.apply(
+          new Union<>(name, dataSets.get(0).getTypeDescriptor()), dataSets);
+    }
+  }
+
+  private Union(@Nullable String name, @Nullable TypeDescriptor<InputT> outputType) {
+    super(name, outputType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java
new file mode 100644
index 00000000000..86d652aec3b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Builders;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Holds state of {@link Window} builder.
+ *
+ * @param <T> type of windowed element
+ */
+class WindowBuilder<T>
+    implements Builders.WindowBy<WindowBuilder<T>>,
+        Builders.TriggeredBy<WindowBuilder<T>>,
+        Builders.AccumulationMode<WindowBuilder<T>>,
+        Builders.WindowedOutput<WindowBuilder<T>> {
+
+  @Nullable private Window<T> window;
+
+  /**
+   * Get underlying window.
+   *
+   * @return maybe window
+   */
+  Optional<Window<T>> getWindow() {
+    return Optional.ofNullable(window);
+  }
+
+  /**
+   * Set underlying window. This is useful for building composite operators, such as {@link
+   * ReduceByKey}.
+   *
+   * @param window the window
+   */
+  void setWindow(Window<T> window) {
+    checkState(this.window == null, "Window is already set.");
+    this.window = window;
+  }
+
+  @Override
+  public <W extends BoundedWindow> WindowBuilder<T> windowBy(WindowFn<Object, W> windowFn) {
+    checkState(window == null, "Window is already set.");
+    window = Window.into(windowFn);
+    return this;
+  }
+
+  @Override
+  public WindowBuilder<T> triggeredBy(Trigger trigger) {
+    window = requireNonNull(window).triggering(trigger);
+    return this;
+  }
+
+  @Override
+  public WindowBuilder<T> accumulationMode(WindowingStrategy.AccumulationMode accumulationMode) {
+    switch (requireNonNull(accumulationMode)) {
+      case DISCARDING_FIRED_PANES:
+        window = requireNonNull(window).discardingFiredPanes();
+        break;
+      case ACCUMULATING_FIRED_PANES:
+        window = requireNonNull(window).accumulatingFiredPanes();
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown accumulation mode [" + accumulationMode + "]");
+    }
+    return this;
+  }
+
+  @Override
+  public WindowBuilder<T> withAllowedLateness(Duration allowedLateness) {
+    window = requireNonNull(window).withAllowedLateness(requireNonNull(allowedLateness));
+    return this;
+  }
+
+  @Override
+  public WindowBuilder<T> withAllowedLateness(
+      Duration allowedLateness, Window.ClosingBehavior closingBehavior) {
+    window =
+        requireNonNull(window)
+            .withAllowedLateness(requireNonNull(allowedLateness), requireNonNull(closingBehavior));
+    return this;
+  }
+
+  @Override
+  public WindowBuilder<T> withTimestampCombiner(TimestampCombiner timestampCombiner) {
+    window = requireNonNull(window).withTimestampCombiner(requireNonNull(timestampCombiner));
+    return this;
+  }
+
+  @Override
+  public WindowBuilder<T> withOnTimeBehavior(Window.OnTimeBehavior behavior) {
+    window = requireNonNull(window).withOnTimeBehavior(requireNonNull(behavior));
+    return this;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java
new file mode 100644
index 00000000000..d2ab7e3f060
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.base;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Common methods used in operator builders.
+ *
+ * <p>They serve several purposes:
+ *
+ * <ul>
+ *   <li>Defines united API among all the {@link Operator Operators}.
+ *   <li>Enables to share related javadoc.
+ *   <li>Allows for mandatory chaining of some builders. See {@link WindowBy}.
+ * </ul>
+ *
+ * <p>For internal usage only.
+ */
+@Audience(Audience.Type.INTERNAL)
+public class Builders {
+
+  /**
+   * Usually the first builder in a chain. It defines an {@link Operator Operator's} input {@link
+   * Dataset}.
+   */
+  public interface Of {
+
+    /**
+     * Specifies the input dataset of the operator.
+     *
+     * @param <InputT> the type of elements in the input dataset
+     * @param input the input dataset to recuce
+     * @return the next builder to complete the setup of the operator
+     */
+    <InputT> Object of(Dataset<InputT> input);
+  }
+
+  /**
+   * Builder which adds a key extractor to the {@link Operator} in focus.
+   *
+   * @param <InputT> type of input elements
+   */
+  public interface KeyBy<InputT> {
+
+    /**
+     * Specifies the function to derive the keys from the operator's input elements.
+     *
+     * @param <K> the type of the extracted key
+     * @param keyExtractor a user defined function to extract keys from the processed input
+     *     dataset's elements
+     * @param keyType {@link TypeDescriptor} of key type {@code <K>}
+     * @return the next builder to complete the setup of the operator
+     */
+    <K> Object keyBy(UnaryFunction<InputT, K> keyExtractor, TypeDescriptor<K> keyType);
+
+    default <K> Object keyBy(UnaryFunction<InputT, K> keyExtractor) {
+      return keyBy(keyExtractor, null);
+    }
+  }
+
+  /**
+   * First windowing builder which starts builders chain defining Beam windowing.
+   *
+   * <p>It consumes {@link WindowFn} and it is followed by {@link TriggeredBy} and {@link
+   * AccumulationMode} builders.
+   *
+   * @param <OutTriggerBuilderT> type of following {@link TriggeredBy} builder.
+   */
+  public interface WindowBy<OutTriggerBuilderT extends TriggeredBy> {
+
+    /**
+     * Specifies the windowing strategy to be applied to the input dataset. Unless the operator is
+     * already preceded by an event time assignment, it will process the input elements in ingestion
+     * time.
+     *
+     * @param <W> the type of the windowing, subclass of {@link BoundedWindow}
+     * @param windowing {@link BoundedWindow} subclass used to represent the windows used by given
+     *     {@link WindowFn}.It represents windowing strategy to apply to the input elements.
+     * @return the next builder to complete the setup of the operator
+     */
+    <W extends BoundedWindow> OutTriggerBuilderT windowBy(WindowFn<Object, W> windowing);
+  }
+
+  /**
+   * Second builder in windowing builders chain. It introduces a {@link Trigger}.
+   *
+   * @param <AccumulationModeBuilderT> following {@link AccumulationMode} builder type
+   */
+  public interface TriggeredBy<AccumulationModeBuilderT extends AccumulationMode> {
+
+    AccumulationModeBuilderT triggeredBy(Trigger trigger);
+  }
+
+  /**
+   * Third and last builder in windowing chain introducing {@link WindowingStrategy.AccumulationMode
+   * accumulation mode}.
+   *
+   * @param <WindowedOutputBuilderT> output builder type
+   */
+  public interface AccumulationMode<WindowedOutputBuilderT> {
+
+    WindowedOutputBuilderT accumulationMode(WindowingStrategy.AccumulationMode accumulationMode);
+
+    default WindowedOutputBuilderT discardingFiredPanes() {
+      return accumulationMode(WindowingStrategy.AccumulationMode.DISCARDING_FIRED_PANES);
+    }
+
+    default WindowedOutputBuilderT accumulatingFiredPanes() {
+      return accumulationMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+    }
+  }
+
+  /**
+   * Builder for window optional parameters
+   *
+   * @param <T> output builder type
+   */
+  public interface WindowedOutput<T extends WindowedOutput<T>> {
+
+    /** {@link Window#withAllowedLateness(Duration)} */
+    T withAllowedLateness(Duration allowedLateness);
+
+    /** {@link Window#withAllowedLateness(Duration, Window.ClosingBehavior)} */
+    T withAllowedLateness(Duration allowedLateness, Window.ClosingBehavior closingBehavior);
+
+    /** {@link Window#withTimestampCombiner(TimestampCombiner)} */
+    T withTimestampCombiner(TimestampCombiner timestampCombiner);
+
+    /** {@link Window#withOnTimeBehavior(Window.OnTimeBehavior)} */
+    T withOnTimeBehavior(Window.OnTimeBehavior behavior);
+  }
+
+  /** Output builder, usually last building step. */
+  public interface Output<T> {
+
+    /**
+     * Finalizes the operator and retrieves its output dataset.
+     *
+     * @param outputHints output dataset description
+     * @return the dataset representing the new operator's output
+     */
+    Dataset<T> output(OutputHint... outputHints);
+  }
+
+  /** Similar to {@link Output}, but it adds method which extracts values from {@link KV}. */
+  public interface OutputValues<K, V> extends Output<KV<K, V>> {
+
+    /**
+     * Finalizes the operator and retrieves its output dataset. Using this output new operator
+     * {@link MapElements} is added to the flow to extract values from pairs.
+     *
+     * @param outputHints output dataset description
+     * @return the dataset representing the new operator's output
+     */
+    Dataset<V> outputValues(OutputHint... outputHints);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Operator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Operator.java
new file mode 100644
index 00000000000..75bcf6c84c8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Operator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.base;
+
+import java.io.Serializable;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** An operator base class. All operators extends this class. */
+@Audience(Audience.Type.INTERNAL)
+public abstract class Operator<OutputT> implements Serializable, TypeAware.Output<OutputT> {
+
+  /** Name of the operator. */
+  @Nullable private final String name;
+
+  /** The type of output elements */
+  @Nullable private final TypeDescriptor<OutputT> outputType;
+
+  protected Operator(@Nullable String name, @Nullable TypeDescriptor<OutputT> outputType) {
+    this.name = name;
+    this.outputType = outputType;
+  }
+
+  /**
+   * Returns user provided name of the operator
+   *
+   * @return maybe name
+   */
+  public final Optional<String> getName() {
+    return Optional.ofNullable(name);
+  }
+
+  @Override
+  public Optional<TypeDescriptor<OutputT>> getOutputType() {
+    return Optional.ofNullable(outputType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/OptionalMethodBuilder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/OptionalMethodBuilder.java
new file mode 100644
index 00000000000..73366467898
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/OptionalMethodBuilder.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.base;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+
+/**
+ * Class to be extended by operator builders that want to make use of `applyIf` call.
+ *
+ * @param <InBuilderT> the class of the builder that extends this class
+ */
+@Audience(Audience.Type.INTERNAL)
+public interface OptionalMethodBuilder<
+    InBuilderT extends OptionalMethodBuilder<InBuilderT, OutBuilderT>, OutBuilderT> {
+
+  /**
+   * Apply given modification to builder when condition evaluates to {@code true}.
+   *
+   * @param cond the condition
+   * @param applyWhenConditionHolds the modification
+   * @return next step builder
+   */
+  OutBuilderT applyIf(boolean cond, UnaryFunction<InBuilderT, OutBuilderT> applyWhenConditionHolds);
+
+  /**
+   * Apply given modifications to builder based on condition.
+   *
+   * @param cond the condition to evaluate
+   * @param applyIfTrue modification to ap ply when {@code cond} evaluates to {@code true}
+   * @param applyIfFalse modification to apply when {@code cond} evaluates to {@code false}
+   * @return next step builder
+   */
+  @SuppressWarnings("unchecked")
+  default OutBuilderT applyIf(
+      boolean cond,
+      UnaryFunction<InBuilderT, OutBuilderT> applyIfTrue,
+      UnaryFunction<InBuilderT, OutBuilderT> applyIfFalse) {
+
+    if (cond) {
+      return applyIfTrue.apply((InBuilderT) this);
+    }
+    return applyIfFalse.apply((InBuilderT) this);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/ShuffleOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/ShuffleOperator.java
new file mode 100644
index 00000000000..f0cb2bcf7a5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/ShuffleOperator.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.base;
+
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAware;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** Operator working on some context. */
+@Audience(Audience.Type.INTERNAL)
+public abstract class ShuffleOperator<InputT, KeyT, OutputT> extends Operator<OutputT>
+    implements TypeAware.Key<KeyT>, TypeAware.Output<OutputT>, WindowAware<InputT> {
+
+  private final UnaryFunction<InputT, KeyT> keyExtractor;
+  @Nullable private final TypeDescriptor<KeyT> keyType;
+  @Nullable private final Window<InputT> window;
+
+  protected ShuffleOperator(
+      @Nullable String name,
+      @Nullable TypeDescriptor<OutputT> outputType,
+      UnaryFunction<InputT, KeyT> keyExtractor,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      @Nullable Window<InputT> windowing) {
+    super(name, outputType);
+    this.keyExtractor = keyExtractor;
+    this.keyType = keyType;
+    this.window = windowing;
+  }
+
+  public UnaryFunction<InputT, KeyT> getKeyExtractor() {
+    return keyExtractor;
+  }
+
+  @Override
+  public Optional<TypeDescriptor<KeyT>> getKeyType() {
+    return Optional.ofNullable(keyType);
+  }
+
+  @Override
+  public Optional<Window<InputT>> getWindow() {
+    return Optional.ofNullable(window);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/WindowAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/WindowAware.java
new file mode 100644
index 00000000000..fad2eb592df
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/WindowAware.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.base;
+
+import java.util.Optional;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.transforms.windowing.Window;
+
+/**
+ * Operator aware of windows.
+ *
+ * @param <T> the type of windowed element
+ */
+@Audience(Audience.Type.INTERNAL)
+public interface WindowAware<T> {
+
+  /**
+   * Get configured window.
+   *
+   * @return window or empty
+   */
+  Optional<Window<T>> getWindow();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/package-info.java
new file mode 100644
index 00000000000..bc754240725
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Collections of operators base classes. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.base;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/ComputationHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/ComputationHint.java
new file mode 100644
index 00000000000..5a3f9006331
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/ComputationHint.java
@@ -0,0 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Extra information for runner about Dataset computation. */
+@Audience(Audience.Type.CLIENT)
+public enum ComputationHint implements OutputHint {
+
+  /** Dataset is expensive to compute. */
+  EXPENSIVE
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java
new file mode 100644
index 00000000000..fdd0ccb42b7
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Specify hints about runtime data characteristics. */
+@Audience(Audience.Type.INTERNAL)
+public interface OutputHint extends Serializable {}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/SizeHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/SizeHint.java
new file mode 100644
index 00000000000..576fe92c194
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/SizeHint.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint;
+
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Extra information for runner about Dataset size. */
+@Audience(Audience.Type.CLIENT)
+public enum SizeHint implements OutputHint {
+  /**
+   * Indicate to runner that dataset can fit in memory and this information could be used for
+   * optimization (e.g. Broadcast hash join).
+   */
+  FITS_IN_MEMORY
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/package-info.java
new file mode 100644
index 00000000000..ee7b941125c
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Hints for optimization in translation process. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/package-info.java
new file mode 100644
index 00000000000..f77faf50909
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Collection of pre-made convenient operators. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAware.java
new file mode 100644
index 00000000000..2bb86b7bf2a
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAware.java
@@ -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.beam.sdk.extensions.euphoria.core.client.type;
+
+import java.util.Optional;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * A collection of interfaces which allows access to {@link TypeDescriptor types} of implementing
+ * {@link Operator Operators} properties.
+ *
+ * <p>Note that there is no input typing interface. That is on purpose since all the transformations
+ * are chained together and type of input(s) equals to types of previous transform output(s). Source
+ * transforms needs to know which type of elements are producing explicitly.
+ */
+public class TypeAware {
+
+  /** Returns {@link TypeDescriptor} of this operator output type. */
+  public interface Output<OutputT> {
+
+    Optional<TypeDescriptor<OutputT>> getOutputType();
+  }
+
+  /** Returns {@link TypeDescriptor} of this operator key type. */
+  public interface Key<KeyT> {
+
+    Optional<TypeDescriptor<KeyT>> getKeyType();
+  }
+
+  /** Returns {@link TypeDescriptor} of this operator value type. */
+  public interface Value<ValueT> {
+
+    Optional<TypeDescriptor<ValueT>> getValueType();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwares.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwares.java
new file mode 100644
index 00000000000..67235cc4734
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwares.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.type;
+
+import java.util.Optional;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+public class TypeAwares {
+
+  private static final TypeDescriptor<Object> OBJECT_TYPE = new TypeDescriptor<Object>() {};
+
+  @SuppressWarnings("unchecked")
+  public static <T> TypeDescriptor<T> orObjects(Optional<TypeDescriptor<T>> maybeType) {
+    return maybeType.orElse((TypeDescriptor) OBJECT_TYPE);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java
new file mode 100644
index 00000000000..c4d9a001ead
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.beam.sdk.extensions.euphoria.core.client.type;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeParameter;
+
+/** A collections of {@link TypeDescriptor} construction methods. */
+public class TypeUtils {
+
+  /**
+   * Creates composite {@link TypeDescriptor} of {@code <KV<K,V>}. Provided that both given
+   * parameters are non null.
+   *
+   * @param key key type descriptor
+   * @param value value type descriptor
+   * @param <K> key tye parameter
+   * @param <V> value type parameter
+   * @return {@link TypeDescriptor} of {@code <KV<K,V>} when {@code key} and {@code value} are not
+   *     null, null otherwise
+   */
+  @Nullable
+  public static <K, V> TypeDescriptor<KV<K, V>> keyValues(
+      TypeDescriptor<K> key, TypeDescriptor<V> value) {
+
+    if (Objects.isNull(key) || Objects.isNull(value)) {
+      return null;
+    }
+
+    return new TypeDescriptor<KV<K, V>>() {}.where(new TypeParameter<K>() {}, key)
+        .where(new TypeParameter<V>() {}, value);
+  }
+
+  /**
+   * Creates composite {@link TypeDescriptor} of {@code <KV<K,V>}. Provided that both given
+   * parameters are non null.
+   *
+   * @param key key type descriptor
+   * @param value value type descriptor
+   * @param <K> key type parameter
+   * @param <V> value type parameter
+   * @return {@link TypeDescriptor} of {@code <KV<K,V>} when {@code key} and {@code value} are not
+   *     null, null otherwise
+   */
+  @Nullable
+  public static <K, V> TypeDescriptor<KV<K, V>> keyValues(Class<K> key, Class<V> value) {
+
+    if (Objects.isNull(key) || Objects.isNull(value)) {
+      return null;
+    }
+
+    return keyValues(TypeDescriptor.of(key), TypeDescriptor.of(value));
+  }
+
+  /**
+   * Creates composite {@link TypeDescriptor} of {@code <Triple<K,V, ScoreT>}. Provided that all
+   * given parameters are non null.
+   *
+   * @param key key type descriptor
+   * @param value value type descriptor
+   * @param score score type descriptor
+   * @param <K> key type parameter
+   * @param <V> value type parameter
+   * @param <ScoreT> score type parameter
+   * @return {@link TypeDescriptor} of {@code <Triple<K,V, ScoreT>} or {@code null} when any given
+   *     parameter is {@code null}
+   */
+  public static <K, V, ScoreT> TypeDescriptor<Triple<K, V, ScoreT>> triplets(
+      TypeDescriptor<K> key, TypeDescriptor<V> value, TypeDescriptor<ScoreT> score) {
+
+    if (Objects.isNull(key) || Objects.isNull(value) || Objects.isNull(score)) {
+      return null;
+    }
+
+    return new TypeDescriptor<Triple<K, V, ScoreT>>() {}.where(new TypeParameter<K>() {}, key)
+        .where(new TypeParameter<V>() {}, value)
+        .where(new TypeParameter<ScoreT>() {}, score);
+  }
+
+  /**
+   * Returns {@link TypeDescriptor} od elements in given {@code dataset} if available, {@code null}
+   * otherwise.
+   *
+   * <p>{@link TypeDescriptor} is acquired as outpyut type of the {@link Operator} which is a
+   * producer of given {@link Dataset}.
+   *
+   * @param <T> type of element of given {@link Dataset}
+   * @param dataset input dataset
+   * @return given dataset element's {@link TypeDescriptor} is available
+   */
+  @Nullable
+  public static <T> TypeDescriptor<T> getDatasetElementType(Dataset<T> dataset) {
+    if (dataset != null && dataset.getProducer().isPresent()) {
+      @SuppressWarnings("unchecked")
+      final Operator<T> operator = (Operator) dataset.getProducer().get();
+      return operator.getOutputType().orElse(null);
+    }
+    return null;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java
new file mode 100644
index 00000000000..cbeae54002b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Typing related utility classes. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.type;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java
new file mode 100644
index 00000000000..6e7066cff07
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.util;
+
+import java.io.Serializable;
+import java.util.stream.Stream;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.SingleValueContext;
+
+/** Apply a folding function. */
+public class Fold implements Serializable {
+
+  /**
+   * Return a {@link CombinableReduceFunction} that performs a fold operation and emits result after
+   * fold of all input data.
+   *
+   * @param <T> element type
+   * @param fold the associative fold function
+   * @return the {@link CombinableReduceFunction}
+   */
+  public static <T> CombinableReduceFunction<T> of(BinaryFunction<T, T, T> fold) {
+    return s ->
+        s.reduce(fold::apply)
+            .orElseThrow(() -> new IllegalStateException("Received empty stream on input!"));
+  }
+
+  /**
+   * Return a {@link CombinableReduceFunction} that performs a fold operation and emits result after
+   * fold of all input data.
+   *
+   * @param <T> element type
+   * @param identity the zero element
+   * @param fold the associative fold function
+   * @return the {@link CombinableReduceFunction}
+   */
+  public static <T> CombinableReduceFunction<T> of(T identity, BinaryFunction<T, T, T> fold) {
+    return s -> s.reduce(identity, fold::apply);
+  }
+
+  /**
+   * Return a {@link ReduceFunctor} that performs a fold operation and emits result after fold of
+   * all input data.
+   *
+   * @param <T> element type
+   * @param identity the zero element
+   * @param fold the associative fold function
+   * @return the {@link CombinableReduceFunction}
+   */
+  public static <T> ReduceFunctor<T, T> of(T identity, BinaryFunctor<T, T, T> fold) {
+    return (Stream<T> s, Collector<T> ctx) -> {
+      final SingleValueContext<T> wrap = new SingleValueContext<>(ctx.asContext());
+      final T ret =
+          s.reduce(
+              identity,
+              (a, b) -> {
+                fold.apply(a, b, wrap);
+                return wrap.getAndResetValue();
+              });
+      ctx.collect(ret);
+    };
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java
new file mode 100644
index 00000000000..44cfd03e357
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.util;
+
+import java.util.Comparator;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+
+/** Calculate maximum. */
+@Audience(Audience.Type.CLIENT)
+public class Max {
+
+  private Max() {}
+
+  /**
+   * Return {@code CombinableReduceFunction} to calculate maximum of input.
+   *
+   * @param <InputT> the type of elements handled
+   * @param <X> the type of key by which to compare the elements
+   * @param extract the key extraction function
+   * @return a combiner function which delivers the "maximum" element seen; never {@code null}
+   */
+  public static <InputT, X extends Comparable<X>> CombinableReduceFunction<InputT> of(
+      UnaryFunction<InputT, X> extract) {
+
+    return values ->
+        values
+            .max(Comparator.comparing(extract::apply))
+            .orElseThrow(() -> new IllegalStateException("Got empty stream on input!"));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java
new file mode 100644
index 00000000000..46cf1d32e70
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.util;
+
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction;
+
+/** Provides commonly used function objects around computing sums. */
+@Audience(Audience.Type.CLIENT)
+public class Sums {
+
+  private static final CombinableReduceFunction<Long> SUMS_OF_LONG =
+      (CombinableReduceFunction<Long>) s -> s.collect(Collectors.summingLong(e -> e));
+  private static final CombinableReduceFunction<Integer> SUMS_OF_INT =
+      (CombinableReduceFunction<Integer>) s -> s.collect(Collectors.summingInt(e -> e));
+
+  private Sums() {}
+
+  public static CombinableReduceFunction<Long> ofLongs() {
+    return SUMS_OF_LONG;
+  }
+
+  public static CombinableReduceFunction<Integer> ofInts() {
+    return SUMS_OF_INT;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java
new file mode 100644
index 00000000000..00191518f70
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.util;
+
+import java.util.Objects;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/** Triple of any types. */
+@Audience(Audience.Type.CLIENT)
+public final class Triple<T1, T2, T3> {
+  final T1 first;
+  final T2 second;
+  final T3 third;
+
+  private Triple(T1 first, T2 second, T3 third) {
+    this.first = first;
+    this.second = second;
+    this.third = third;
+  }
+
+  public static <T1, T2, T3> Triple<T1, T2, T3> of(T1 first, T2 second, T3 third) {
+    return new Triple<>(first, second, third);
+  }
+
+  public T1 getFirst() {
+    return first;
+  }
+
+  public T2 getSecond() {
+    return second;
+  }
+
+  public T3 getThird() {
+    return third;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof Triple) {
+      Triple<?, ?, ?> triple = (Triple<?, ?, ?>) o;
+      return Objects.equals(first, triple.first)
+          && Objects.equals(second, triple.second)
+          && Objects.equals(third, triple.third);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(first, second, third);
+  }
+
+  @Override
+  public String toString() {
+    return "Triple{" + "first=" + first + ", second=" + second + ", third=" + third + '}';
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java
new file mode 100644
index 00000000000..2aef1d8e0ad
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Client (user) facing utility classes. */
+package org.apache.beam.sdk.extensions.euphoria.core.client.util;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/EuphoriaCoderProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/EuphoriaCoderProvider.java
new file mode 100644
index 00000000000..48d06e39fe3
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/EuphoriaCoderProvider.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import com.esotericsoftware.kryo.ClassResolver;
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Registration;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderProvider;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of {@link CoderProvider}, backbone of {@link RegisterCoders} API and {@link Kryo}
+ * integration.
+ */
+public class EuphoriaCoderProvider extends CoderProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(EuphoriaCoderProvider.class);
+
+  private final Map<TypeDescriptor, Coder<?>> typeToCoder;
+  private final Map<Class<?>, Coder<?>> classToCoder;
+  private final IdentifiedRegistrar kryoRegistrar;
+
+  EuphoriaCoderProvider(
+      Map<TypeDescriptor, Coder<?>> typeToCoder,
+      Map<Class<?>, Coder<?>> classToCoder,
+      KryoRegistrar kryoRegistrar) {
+    this.typeToCoder = typeToCoder;
+    this.classToCoder = classToCoder;
+    if (kryoRegistrar != null) {
+      this.kryoRegistrar = IdentifiedRegistrar.of(kryoRegistrar);
+    } else {
+      this.kryoRegistrar = null;
+    }
+  }
+
+  @Override
+  public <T> Coder<T> coderFor(
+      TypeDescriptor<T> typeDescriptor, List<? extends Coder<?>> componentCoders)
+      throws CannotProvideCoderException {
+
+    // try to obtain most specific coder by type descriptor
+    Coder<?> coder = typeToCoder.get(typeDescriptor);
+
+    // second try, obtain coder by raw encoding type
+    if (coder == null) {
+      Class<? super T> rawType = typeDescriptor.getRawType();
+      coder = classToCoder.get(rawType);
+
+      // if we still do not have a coder check whenever given class was registered with kryo
+      if (coder == null) {
+        coder = createKryoCoderIfClassRegistered(rawType);
+      }
+    }
+
+    if (coder == null) {
+      LOG.info(String.format("%s cannot provide coder for '%s'", this, typeDescriptor));
+      throw new CannotProvideCoderException(
+          String.format("No coder for given type descriptor '%s' found.", typeDescriptor));
+    }
+
+    @SuppressWarnings("unchecked")
+    Coder<T> castedCoder = (Coder<T>) coder;
+
+    return castedCoder;
+  }
+
+  private <T> Coder<T> createKryoCoderIfClassRegistered(Class<? super T> rawType) {
+
+    if (kryoRegistrar == null) {
+      return null;
+    }
+
+    Kryo kryo = KryoFactory.getOrCreateKryo(kryoRegistrar);
+    ClassResolver classResolver = kryo.getClassResolver();
+
+    Registration registration = classResolver.getRegistration(rawType);
+    if (registration == null) {
+      return null;
+    }
+
+    Coder<T> coder = createKryoCoderWithRegisteredClasses();
+    classToCoder.put(rawType, coder);
+
+    return coder;
+  }
+
+  public <T> KryoCoder<T> createKryoCoderWithRegisteredClasses() {
+    return KryoCoder.of(kryoRegistrar);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/IdentifiedRegistrar.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/IdentifiedRegistrar.java
new file mode 100644
index 00000000000..740bc50bee7
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/IdentifiedRegistrar.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import com.esotericsoftware.kryo.Kryo;
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link KryoRegistrar} enriched by Id.
+ *
+ * <p>New instances of the same (possibly lambda) implementation of {@link KryoRegistrar} may be
+ * created by (de)serialization. And since lambda expressions do not retain their type (instance of
+ * {@link Class}) after deserialization, we need something else to avoid creation of more {@link
+ * Kryo} instances then really needed. That is why any given {@link KryoRegistrar} instance is
+ * enriched by Id.
+ */
+class IdentifiedRegistrar implements Serializable {
+
+  static final int NO_OP_REGISTRAR_ID = -1;
+  private static final Logger LOG = LoggerFactory.getLogger(RegisterCoders.class);
+
+  private static final AtomicInteger idSource = new AtomicInteger();
+
+  private final int id;
+  private final KryoRegistrar registrar;
+
+  private IdentifiedRegistrar(int id, KryoRegistrar registrar) {
+    this.id = id;
+    this.registrar = registrar;
+  }
+
+  static IdentifiedRegistrar of(KryoRegistrar registrar) {
+    Objects.requireNonNull(registrar);
+    IdentifiedRegistrar identifiedRegistrar =
+        new IdentifiedRegistrar(idSource.getAndIncrement(), registrar);
+    LOG.debug(
+        "Registrar id [{}] was assigned to [{}].",
+        identifiedRegistrar.getId(),
+        registrar.getClass());
+    return identifiedRegistrar;
+  }
+
+  static IdentifiedRegistrar defaultNoOpRegistrar() {
+    return new IdentifiedRegistrar(
+        NO_OP_REGISTRAR_ID,
+        (kryo) -> {
+          /*No-Op*/
+        });
+  }
+
+  @Override
+  public String toString() {
+    return "IdentifiedRegistrar{" + "id=" + id + ", registrar=" + registrar + '}';
+  }
+
+  public int getId() {
+    return id;
+  }
+
+  public KryoRegistrar getRegistrar() {
+    return registrar;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoCoder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoCoder.java
new file mode 100644
index 00000000000..a5a21e7cc7b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoCoder.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoException;
+import com.esotericsoftware.kryo.io.InputChunked;
+import com.esotericsoftware.kryo.io.OutputChunked;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+
+/**
+ * Coder using Kryo as (de)serialization mechanism. See {@link RegisterCoders} to get more details
+ * of how to use it
+ */
+public class KryoCoder<T> extends CustomCoder<T> {
+
+  /**
+   * Client-defined class registrations to {@link Kryo}.
+   *
+   * <p>{@link KryoCoder} needs it to be able to create a {@link Kryo} instance with correct class
+   * registrations after its deserialization.
+   */
+  private final IdentifiedRegistrar registrarWithId;
+
+  private KryoCoder(IdentifiedRegistrar registrarWithId) {
+    this.registrarWithId = registrarWithId;
+  }
+
+  /**
+   * @param <T> type of element this class should code/encode
+   * @param registrarWithId uniquely identified {@link KryoRegistrar} which is used to register
+   *     classes to {@link Kryo} instance used by returned {@link KryoCoder}
+   * @return Newly created a {@link KryoCoder} instance which will use {@link Kryo} with classes
+   *     registered by {@code registrarWithId}.
+   */
+  public static <T> KryoCoder<T> of(IdentifiedRegistrar registrarWithId) {
+    return new KryoCoder<>(registrarWithId);
+  }
+
+  /**
+   * @param <T> type of element this class should code/encode
+   * @return Newly created a {@link KryoCoder} instance which will use {@link Kryo} without class
+   *     registration. That degrades performance. Use {@link #of(IdentifiedRegistrar)} whenever
+   *     possible.
+   */
+  public static <T> KryoCoder<T> withoutClassRegistration() {
+    return new KryoCoder<>(KryoFactory.NO_OP_REGISTRAR);
+  }
+
+  @Override
+  public void encode(T value, OutputStream outStream) throws IOException {
+
+    Kryo kryo = KryoFactory.getOrCreateKryo(registrarWithId);
+
+    OutputChunked output = KryoFactory.getKryoOutput();
+    output.clear();
+    output.setOutputStream(outStream);
+
+    try {
+      kryo.writeClassAndObject(output, value);
+      output.endChunks();
+      output.flush();
+    } catch (IllegalArgumentException e) {
+      throw new CoderException(
+          String.format(
+              "Cannot encode given object of type '%s'. "
+                  + "Forgotten kryo registration is possible explanation. Kryo registrations where done by '%s'.",
+              (value == null) ? null : value.getClass().getSimpleName(), registrarWithId),
+          e);
+    }
+  }
+
+  @Override
+  public T decode(InputStream inStream) throws IOException {
+
+    InputChunked input = KryoFactory.getKryoInput();
+    input.rewind();
+    input.setInputStream(inStream);
+
+    Kryo kryo = KryoFactory.getOrCreateKryo(registrarWithId);
+
+    try {
+      @SuppressWarnings("unchecked")
+      T outObject = (T) kryo.readClassAndObject(input);
+      return outObject;
+
+    } catch (KryoException e) {
+      throw new CoderException(
+          String.format(
+              "Cannot decode object from input stream."
+                  + " Forgotten kryo registration is possible explanation. Kryo registrations where done by '%s'.",
+              registrarWithId),
+          e);
+    }
+  }
+
+  @Override
+  public void verifyDeterministic() throws NonDeterministicException {
+    // nop
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoFactory.java
new file mode 100644
index 00000000000..cdb0f7217e1
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoFactory.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.InputChunked;
+import com.esotericsoftware.kryo.io.Output;
+import com.esotericsoftware.kryo.io.OutputChunked;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import org.objenesis.strategy.StdInstantiatorStrategy;
+
+/**
+ * A source of {@link Kryo} instances. It allows {@link Kryo} to be reused by many {@link KryoCoder
+ * KryoCoders}.
+ */
+class KryoFactory {
+
+  /** Initial size of byte buffers in {@link Output}, {@link Input}. */
+  private static final int DEFAULT_BUFFER_SIZE = 4096;
+
+  /**
+   * No-op {@link IdentifiedRegistrar}. Use of this registrar degrades performance since {@link
+   * Kryo} needs to serialize fully specified class name instead of id.
+   *
+   * <p>{@link #getOrCreateKryo(IdentifiedRegistrar)} returns {@link Kryo} which allows for
+   * serialization of unregistered classes when this {@link IdentifiedRegistrar} is used to call it.
+   */
+  static final IdentifiedRegistrar NO_OP_REGISTRAR = IdentifiedRegistrar.defaultNoOpRegistrar();
+
+  private static Kryo createKryo(IdentifiedRegistrar registrarWithId) {
+    final Kryo instance = new Kryo();
+    ((Kryo.DefaultInstantiatorStrategy) instance.getInstantiatorStrategy())
+        .setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
+
+    // mere NO_OP_REGISTRAR == registrarWithId is not enough since
+    // NO_OP_REGISTRAR can be deserialized into several instances
+    if (registrarWithId.getId() == IdentifiedRegistrar.NO_OP_REGISTRAR_ID) {
+      instance.setRegistrationRequired(false);
+    } else {
+      instance.setRegistrationRequired(true);
+      registrarWithId.getRegistrar().registerClasses(instance);
+    }
+
+    return instance;
+  }
+
+  private static ThreadLocal<OutputChunked> threadLocalOutput =
+      ThreadLocal.withInitial(() -> new OutputChunked(DEFAULT_BUFFER_SIZE));
+
+  private static ThreadLocal<InputChunked> threadLocalInput =
+      ThreadLocal.withInitial(() -> new InputChunked(DEFAULT_BUFFER_SIZE));
+
+  /**
+   * We need an instance of {@link KryoRegistrar} to do actual {@link Kryo} registration. But since
+   * every other instance of the same implementation of {@link KryoRegistrar} should do the same
+   * classes registration, we use {@link IdentifiedRegistrar IdentifiedRegistrar's} Id as a key.
+   *
+   * <p>{@link ThreadLocal} is utilized to allow re-usability of {@link Kryo} by many instances of
+   * {@link KryoCoder}.
+   */
+  private static Map<Integer, ThreadLocal<Kryo>> kryoByRegistrarId = new HashMap<>();
+
+  /**
+   * Returns {@link Kryo} instance which has classes registered by this {@code registrar} or
+   * previously given {@link KryoRegistrar} instance of the same type. The returned instance is
+   * either created by this call or returned from cache.
+   *
+   * <p>If given {@code registrar} is {@link #NO_OP_REGISTRAR} then returned kryo allows for
+   * (de)serialization of unregistered classes. That is not otherwise allowed.
+   */
+  static Kryo getOrCreateKryo(IdentifiedRegistrar registrarWithId) {
+    Objects.requireNonNull(registrarWithId);
+
+    synchronized (kryoByRegistrarId) {
+      ThreadLocal<Kryo> kryoThreadLocal =
+          kryoByRegistrarId.computeIfAbsent(registrarWithId.getId(), (k) -> new ThreadLocal<>());
+
+      Kryo kryoInstance = kryoThreadLocal.get();
+      if (kryoInstance == null) {
+        kryoInstance = createKryo(registrarWithId);
+        kryoThreadLocal.set(kryoInstance);
+      }
+
+      return kryoInstance;
+    }
+  }
+
+  static InputChunked getKryoInput() {
+    return threadLocalInput.get();
+  }
+
+  static OutputChunked getKryoOutput() {
+    return threadLocalOutput.get();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoRegistrar.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoRegistrar.java
new file mode 100644
index 00000000000..6541cb86140
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoRegistrar.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import com.esotericsoftware.kryo.Kryo;
+import java.io.Serializable;
+
+/**
+ * Interface to be implemented by clients to register their classes with {@link Kryo}.
+ *
+ * <p>{@link KryoRegistrar KryoRegistrars} have to be stateless and {@link Serializable}.
+ */
+public interface KryoRegistrar extends Serializable {
+
+  /**
+   * Implementations should call variants of {@link Kryo#register(Class)} to register custom classes
+   * with given {@link Kryo} instance. It should be stateless and resulting in the same type
+   * registrations (including order of registration) every time it is called.
+   *
+   * @param kryo {@link Kryo} instance to be used for type registration
+   */
+  void registerClasses(Kryo kryo);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/RegisterCoders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/RegisterCoders.java
new file mode 100644
index 00000000000..008cd1239b2
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/RegisterCoders.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import static java.util.Objects.requireNonNull;
+
+import com.esotericsoftware.kryo.Kryo;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.stability.Experimental;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** Convenient way of registering Beam {@link Coder}s to the given {@link Pipeline}. */
+@Experimental
+public class RegisterCoders {
+
+  public static KryoBuilder to(Pipeline pipeline) {
+    return new Builder(requireNonNull(pipeline));
+  }
+
+  // ----------------------------- builder chain
+
+  /** Builder which allows to se {@link KryoRegistrar}. */
+  public interface KryoBuilder extends RegisterBuilder {
+
+    /**
+     * Sets {@link KryoRegistrar}. All the classes registered by it are automatically coded using
+     * {@link Kryo}.
+     *
+     * @param registrar user defined class registrations to {@link Kryo}
+     * @return {@link RegisterBuilder} which allows user to register coders of its own
+     */
+    RegisterBuilder setKryoClassRegistrar(KryoRegistrar registrar);
+  }
+
+  /** Builder which defines all non {@link com.esotericsoftware.kryo.Kryo} registration methods. */
+  public interface RegisterBuilder {
+
+    /**
+     * Registers custom {@link Coder} for given parametrized {@link TypeDescriptor}.
+     *
+     * @param type type to register coder for
+     * @param coder coder to register
+     * @param <T> type of elements encoded by given {@code coder}
+     * @return {@link RegisterBuilder} to allow for more coders registration.
+     */
+    <T> RegisterBuilder registerCoder(TypeDescriptor<T> type, Coder<T> coder);
+
+    /**
+     * Registers custom {@link Coder} for given raw {@link Class type}.
+     *
+     * @param clazz type to register coder for
+     * @param coder coder to register
+     * @param <T> type of elements encoded by given {@code coder}
+     * @return {@link RegisterBuilder} to allow for more coders registration.
+     */
+    <T> RegisterBuilder registerCoder(Class<T> clazz, Coder<T> coder);
+
+    /** Effectively ends coders registration. No coders registration is done without it. */
+    void done();
+  }
+
+  // ----------------------------- builder itself
+
+  /** Builder of {@link RegisterCoders}. */
+  public static class Builder implements RegisterBuilder, KryoBuilder {
+
+    private final Pipeline pipeline;
+    private final Map<TypeDescriptor, Coder<?>> typeToCoder = new HashMap<>();
+    private final Map<Class<?>, Coder<?>> classToCoder = new HashMap<>();
+    private KryoRegistrar registrar;
+
+    Builder(Pipeline pipeline) {
+      this.pipeline = pipeline;
+    }
+
+    @Override
+    public <T> RegisterBuilder registerCoder(TypeDescriptor<T> type, Coder<T> coder) {
+      requireNonNull(type);
+      requireNonNull(coder);
+      typeToCoder.put(type, coder);
+      return this;
+    }
+
+    @Override
+    public <T> RegisterBuilder registerCoder(Class<T> clazz, Coder<T> coder) {
+      requireNonNull(clazz);
+      requireNonNull(coder);
+      classToCoder.put(clazz, coder);
+      return this;
+    }
+
+    @Override
+    public RegisterBuilder setKryoClassRegistrar(KryoRegistrar registrar) {
+      requireNonNull(registrar);
+      this.registrar = registrar;
+      return this;
+    }
+
+    @Override
+    public void done() {
+      pipeline
+          .getCoderRegistry()
+          .registerCoderProvider(new EuphoriaCoderProvider(typeToCoder, classToCoder, registrar));
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/package-info.java
new file mode 100644
index 00000000000..90373db40ac
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/coder/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Euphoria API focused on Beam coders. */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/AbstractJoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/AbstractJoinTranslator.java
new file mode 100644
index 00000000000..d50f2fecb99
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/AbstractJoinTranslator.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+abstract class AbstractJoinTranslator<LeftT, RightT, KeyT, OutputT>
+    implements OperatorTranslator<Object, KV<KeyT, OutputT>, Join<LeftT, RightT, KeyT, OutputT>> {
+
+  @Override
+  public PCollection<KV<KeyT, OutputT>> translate(
+      Join<LeftT, RightT, KeyT, OutputT> operator, PCollectionList<Object> inputs) {
+    checkArgument(inputs.size() == 2, "Join expects exactly two inputs.");
+    @SuppressWarnings("unchecked")
+    final PCollection<LeftT> left = (PCollection) inputs.get(0);
+    @SuppressWarnings("unchecked")
+    final PCollection<RightT> right = (PCollection) inputs.get(1);
+    PCollection<KV<KeyT, LeftT>> leftKeyed =
+        left.apply(
+            "extract-keys-left",
+            new ExtractKey<>(
+                operator.getLeftKeyExtractor(), TypeAwares.orObjects(operator.getKeyType())));
+    PCollection<KV<KeyT, RightT>> rightKeyed =
+        right.apply(
+            "extract-keys-right",
+            new ExtractKey<>(
+                operator.getRightKeyExtractor(), TypeAwares.orObjects(operator.getKeyType())));
+    // apply windowing if specified
+    if (operator.getWindow().isPresent()) {
+      @SuppressWarnings("unchecked")
+      final Window<KV<KeyT, LeftT>> leftWindow = (Window) operator.getWindow().get();
+      leftKeyed = leftKeyed.apply("window-left", leftWindow);
+      @SuppressWarnings("unchecked")
+      final Window<KV<KeyT, RightT>> rightWindow = (Window) operator.getWindow().get();
+      rightKeyed = rightKeyed.apply("window-right", rightWindow);
+    }
+    return translate(operator, leftKeyed, rightKeyed)
+        .setTypeDescriptor(
+            operator
+                .getOutputType()
+                .orElseThrow(
+                    () -> new IllegalStateException("Unable to infer output type descriptor.")));
+  }
+
+  abstract PCollection<KV<KeyT, OutputT>> translate(
+      Join<LeftT, RightT, KeyT, OutputT> operator,
+      PCollection<KV<KeyT, LeftT>> left,
+      PCollection<KV<KeyT, RightT>> right);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BeamAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BeamAccumulatorProvider.java
new file mode 100644
index 00000000000..12d5b61e09f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BeamAccumulatorProvider.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Translation of accumulators to {@link Metrics}. Metric's namespace is taken from operator name.
+ * So for better orientation in metrics it's recommended specify operator name with method .named().
+ */
+public class BeamAccumulatorProvider implements AccumulatorProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BeamAccumulatorProvider.class);
+  private static final String KEY_METRIC_SEPARATOR = "::";
+
+  private final Map<String, Counter> counterMap = new ConcurrentHashMap<>();
+  private final Map<String, Histogram> histogramMap = new ConcurrentHashMap<>();
+
+  private BeamAccumulatorProvider() {}
+
+  public static Factory getFactory() {
+    return Factory.get();
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    throw new UnsupportedOperationException(
+        "BeamAccumulatorProvider doesn't support"
+            + " getCounter(String name). Please specify namespace and name.");
+  }
+
+  @Override
+  public Counter getCounter(final String namespace, final String name) {
+    return counterMap.computeIfAbsent(
+        getMetricsKey(namespace, name), key -> new BeamMetricsCounter(namespace, name));
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    throw new UnsupportedOperationException(
+        "BeamAccumulatorProvider doesn't support"
+            + " getHistogram(String name). Please specify namespace and name.");
+  }
+
+  @Override
+  public Histogram getHistogram(final String namespace, final String name) {
+    return histogramMap.computeIfAbsent(
+        getMetricsKey(namespace, name), key -> new BeamMetricsHistogram(namespace, name));
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    throw new UnsupportedOperationException(
+        "BeamAccumulatorProvider doesn't support"
+            + " getTimer(String name). Please specify namespace and name.");
+  }
+
+  /**
+   * Metric key for accumulator map.
+   *
+   * @param namespace = operator name
+   * @param name of metric
+   * @return metricKey = namespace + SEPARATOR + name
+   */
+  private static String getMetricsKey(String namespace, String name) {
+    return namespace.concat(KEY_METRIC_SEPARATOR).concat(name);
+  }
+
+  // ------------------------
+
+  /** AccumulatorProvider Factory. */
+  public static class Factory implements AccumulatorProvider.Factory {
+
+    private static final BeamAccumulatorProvider.Factory INSTANCE =
+        new BeamAccumulatorProvider.Factory();
+
+    private static final AccumulatorProvider PROVIDER = new BeamAccumulatorProvider();
+
+    private static final AtomicBoolean isLogged = new AtomicBoolean();
+
+    private Factory() {}
+
+    public static BeamAccumulatorProvider.Factory get() {
+      return INSTANCE;
+    }
+
+    @Override
+    public AccumulatorProvider create() {
+      if (isLogged.compareAndSet(false, true)) {
+        LOG.info("Using accumulators with BeamAccumulatorProvider");
+      }
+      return PROVIDER;
+    }
+  }
+
+  // ------------------------
+
+  /** Implementation of Counter via {@link org.apache.beam.sdk.metrics.Counter}. */
+  public static class BeamMetricsCounter extends BeamMetrics implements Counter {
+
+    BeamMetricsCounter(String namespace, String name) {
+      super(namespace, name);
+    }
+
+    @Override
+    public void increment(long value) {
+      Metrics.counter(getNamespace(), getName()).inc(value);
+    }
+
+    @Override
+    public void increment() {
+      Metrics.counter(getNamespace(), getName()).inc();
+    }
+  }
+
+  /** Implementation of Histogram via {@link org.apache.beam.sdk.metrics.Distribution}. */
+  public static class BeamMetricsHistogram extends BeamMetrics implements Histogram {
+
+    BeamMetricsHistogram(String namespace, String name) {
+      super(namespace, name);
+    }
+
+    @Override
+    public void add(long value) {
+      Metrics.distribution(getNamespace(), getName()).update(value);
+    }
+
+    @Override
+    public void add(long value, long times) {
+      final Distribution histogram = Metrics.distribution(getNamespace(), getName());
+      for (long i = 0; i < times; i++) {
+        histogram.update(value);
+      }
+    }
+  }
+
+  private abstract static class BeamMetrics {
+
+    private final String namespace;
+    private final String name;
+
+    BeamMetrics(String namespace, String name) {
+      this.namespace = namespace;
+      this.name = name;
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public String getNamespace() {
+      return namespace;
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java
new file mode 100644
index 00000000000..90870a13e4d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import java.util.Collections;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.collector.AdaptableCollector;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+/**
+ * Translator for {@link org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin} and
+ * {@link org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin} when one side of
+ * the join fits in memory so it can be distributed in hash map with the other side.
+ */
+public class BroadcastHashJoinTranslator<LeftT, RightT, KeyT, OutputT>
+    extends AbstractJoinTranslator<LeftT, RightT, KeyT, OutputT> {
+
+  @Override
+  PCollection<KV<KeyT, OutputT>> translate(
+      Join<LeftT, RightT, KeyT, OutputT> operator,
+      PCollection<KV<KeyT, LeftT>> left,
+      PCollection<KV<KeyT, RightT>> right) {
+    final AccumulatorProvider accumulators =
+        new LazyAccumulatorProvider(AccumulatorProvider.of(left.getPipeline()));
+    switch (operator.getType()) {
+      case LEFT:
+        final PCollectionView<Map<KeyT, Iterable<RightT>>> broadcastRight =
+            right.apply(View.asMultimap());
+        return left.apply(
+            ParDo.of(
+                    new BroadcastHashLeftJoinFn<>(
+                        broadcastRight,
+                        operator.getJoiner(),
+                        accumulators,
+                        operator.getName().orElse(null)))
+                .withSideInputs(broadcastRight));
+      case RIGHT:
+        final PCollectionView<Map<KeyT, Iterable<LeftT>>> broadcastLeft =
+            left.apply(View.asMultimap());
+        return right.apply(
+            ParDo.of(
+                    new BroadcastHashRightJoinFn<>(
+                        broadcastLeft,
+                        operator.getJoiner(),
+                        accumulators,
+                        operator.getName().orElse(null)))
+                .withSideInputs(broadcastLeft));
+      default:
+        throw new UnsupportedOperationException(
+            String.format(
+                "Cannot translate Euphoria '%s' operator to Beam transformations."
+                    + " Given join type '%s' is not supported for BroadcastHashJoin.",
+                Join.class.getSimpleName(), operator.getType()));
+    }
+  }
+
+  static class BroadcastHashRightJoinFn<K, LeftT, RightT, OutputT>
+      extends DoFn<KV<K, RightT>, KV<K, OutputT>> {
+
+    private final PCollectionView<Map<K, Iterable<LeftT>>> smallSideCollection;
+    private final BinaryFunctor<LeftT, RightT, OutputT> joiner;
+    private final AdaptableCollector<KV<K, RightT>, KV<K, OutputT>, OutputT> outCollector;
+
+    BroadcastHashRightJoinFn(
+        PCollectionView<Map<K, Iterable<LeftT>>> smallSideCollection,
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        AccumulatorProvider accumulators,
+        @Nullable String operatorName) {
+      this.smallSideCollection = smallSideCollection;
+      this.joiner = joiner;
+      this.outCollector =
+          new AdaptableCollector<>(
+              accumulators,
+              operatorName,
+              (ctx, elem) -> ctx.output(KV.of(ctx.element().getKey(), elem)));
+    }
+
+    @SuppressWarnings("unused")
+    @ProcessElement
+    public void processElement(ProcessContext context) {
+      final KV<K, RightT> element = context.element();
+      final K key = element.getKey();
+      final Map<K, Iterable<LeftT>> map = context.sideInput(smallSideCollection);
+      final Iterable<LeftT> leftValues = map.getOrDefault(key, Collections.singletonList(null));
+      outCollector.setProcessContext(context);
+      leftValues.forEach(leftValue -> joiner.apply(leftValue, element.getValue(), outCollector));
+    }
+  }
+
+  static class BroadcastHashLeftJoinFn<K, LeftT, RightT, OutputT>
+      extends DoFn<KV<K, LeftT>, KV<K, OutputT>> {
+
+    private final PCollectionView<Map<K, Iterable<RightT>>> smallSideCollection;
+    private final BinaryFunctor<LeftT, RightT, OutputT> joiner;
+    private final AdaptableCollector<KV<K, LeftT>, KV<K, OutputT>, OutputT> outCollector;
+
+    BroadcastHashLeftJoinFn(
+        PCollectionView<Map<K, Iterable<RightT>>> smallSideCollection,
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        AccumulatorProvider accumulators,
+        @Nullable String operatorName) {
+      this.smallSideCollection = smallSideCollection;
+      this.joiner = joiner;
+      this.outCollector =
+          new AdaptableCollector<>(
+              accumulators,
+              operatorName,
+              (ctx, elem) -> ctx.output(KV.of(ctx.element().getKey(), elem)));
+    }
+
+    @SuppressWarnings("unused")
+    @ProcessElement
+    public void processElement(ProcessContext context) {
+      final KV<K, LeftT> element = context.element();
+      final K key = element.getKey();
+      final Map<K, Iterable<RightT>> map = context.sideInput(smallSideCollection);
+      final Iterable<RightT> rightValues = map.getOrDefault(key, Collections.singletonList(null));
+      outCollector.setProcessContext(context);
+      rightValues.forEach(rightValue -> joiner.apply(element.getValue(), rightValue, outCollector));
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/EuphoriaOptions.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/EuphoriaOptions.java
new file mode 100644
index 00000000000..98f76bcbfb4
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/EuphoriaOptions.java
@@ -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.beam.sdk.extensions.euphoria.core.translate;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Pipeline options related to Euphoria DSL translation. */
+public interface EuphoriaOptions extends PipelineOptions {
+
+  class DefaultTranslatorProviderFactory implements DefaultValueFactory<TranslatorProvider> {
+
+    @Override
+    public TranslatorProvider create(PipelineOptions options) {
+      return SimpleTranslatorProvider.newBuilder()
+          .registerTranslator(FlatMap.class, new FlatMapTranslator<>())
+          .registerTranslator(Union.class, new UnionTranslator<>())
+          .registerTranslator(ReduceByKey.class, new ReduceByKeyTranslator<>())
+          .registerTranslator(Join.class, new JoinTranslator<>())
+          .build();
+    }
+  }
+
+  class DefaultAccumulatorProviderFactory
+      implements DefaultValueFactory<AccumulatorProvider.Factory> {
+
+    @Override
+    public AccumulatorProvider.Factory create(PipelineOptions options) {
+      return BeamAccumulatorProvider.Factory.get();
+    }
+  }
+
+  @Description("Euphoria translation provider")
+  @Default.InstanceFactory(DefaultTranslatorProviderFactory.class)
+  @JsonIgnore
+  TranslatorProvider getTranslatorProvider();
+
+  void setTranslatorProvider(TranslatorProvider translationProvider);
+
+  @Description("Euphoria accumulator provider factory")
+  @Default.InstanceFactory(DefaultAccumulatorProviderFactory.class)
+  @JsonIgnore
+  AccumulatorProvider.Factory getAccumulatorProviderFactory();
+
+  void setAccumulatorProviderFactory(AccumulatorProvider.Factory accumulatorProviderFactory);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java
new file mode 100644
index 00000000000..dbd71a412d2
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+
+/** Shared utility methods among operator translators. */
+class ExtractKey<KeyT, ValueT>
+    extends PTransform<PCollection<ValueT>, PCollection<KV<KeyT, ValueT>>> {
+
+  private static class ExtractKeyFn<KeyT, InputT> extends DoFn<InputT, KV<KeyT, InputT>> {
+
+    private final UnaryFunction<InputT, KeyT> keyExtractor;
+
+    ExtractKeyFn(UnaryFunction<InputT, KeyT> keyExtractor) {
+      this.keyExtractor = keyExtractor;
+    }
+
+    @SuppressWarnings("unused")
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      final InputT element = c.element();
+      final KeyT key = keyExtractor.apply(element);
+      c.output(KV.of(key, element));
+    }
+  }
+
+  private final UnaryFunction<ValueT, KeyT> keyExtractor;
+  private final TypeDescriptor<KeyT> keyType;
+
+  ExtractKey(UnaryFunction<ValueT, KeyT> keyExtractor, TypeDescriptor<KeyT> keyType) {
+    this.keyExtractor = requireNonNull(keyExtractor);
+    this.keyType = requireNonNull(keyType);
+  }
+
+  @Override
+  public PCollection<KV<KeyT, ValueT>> expand(PCollection<ValueT> input) {
+    return input
+        .apply(ParDo.of(new ExtractKeyFn<>(keyExtractor)))
+        .setTypeDescriptor(TypeDescriptors.kvs(keyType, input.getTypeDescriptor()));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java
new file mode 100644
index 00000000000..2de54aa51b3
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.collector.AdaptableCollector;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.collector.CollectorAdapter;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.joda.time.Instant;
+
+/**
+ * Default translator for {@link FlatMap} operator.
+ *
+ * @param <InputT> type of input
+ * @param <OutputT> type of output
+ */
+public class FlatMapTranslator<InputT, OutputT>
+    implements OperatorTranslator<InputT, OutputT, FlatMap<InputT, OutputT>> {
+
+  @Override
+  public PCollection<OutputT> translate(
+      FlatMap<InputT, OutputT> operator, PCollectionList<InputT> inputs) {
+    final AccumulatorProvider accumulators =
+        new LazyAccumulatorProvider(AccumulatorProvider.of(inputs.getPipeline()));
+    final Mapper<InputT, OutputT> mapper =
+        new Mapper<>(
+            operator.getName().orElse(null),
+            operator.getFunctor(),
+            accumulators,
+            operator.getEventTimeExtractor().orElse(null));
+    return OperatorTranslators.getSingleInput(inputs)
+        .apply("mapper", ParDo.of(mapper))
+        .setTypeDescriptor(TypeAwares.orObjects(operator.getOutputType()));
+  }
+
+  private static class Mapper<InputT, OutputT> extends DoFn<InputT, OutputT> {
+
+    private final UnaryFunctor<InputT, OutputT> mapper;
+    private final AdaptableCollector<InputT, OutputT, OutputT> collector;
+
+    Mapper(
+        @Nullable String operatorName,
+        UnaryFunctor<InputT, OutputT> mapper,
+        AccumulatorProvider accumulators,
+        @Nullable ExtractEventTime<InputT> eventTimeExtractor) {
+      this.mapper = mapper;
+      this.collector =
+          new AdaptableCollector<>(accumulators, operatorName, new Collector<>(eventTimeExtractor));
+    }
+
+    @ProcessElement
+    @SuppressWarnings("unused")
+    public void processElement(ProcessContext ctx) {
+      collector.setProcessContext(ctx);
+      mapper.apply(ctx.element(), collector);
+    }
+  }
+
+  private static class Collector<InputT, OutputT>
+      implements CollectorAdapter<InputT, OutputT, OutputT> {
+
+    @Nullable private final ExtractEventTime<InputT> eventTimeExtractor;
+
+    private Collector(@Nullable ExtractEventTime<InputT> eventTimeExtractor) {
+      this.eventTimeExtractor = eventTimeExtractor;
+    }
+
+    @Override
+    public void collect(DoFn<InputT, OutputT>.ProcessContext ctx, OutputT out) {
+      if (eventTimeExtractor != null) {
+        InputT element = ctx.element();
+        ctx.outputWithTimestamp(out, new Instant(eventTimeExtractor.extractTimestamp(element)));
+      } else {
+        ctx.output(out);
+      }
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java
new file mode 100644
index 00000000000..09a22dfa8e6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import static java.util.Objects.requireNonNull;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.collector.AdaptableCollector;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+
+/** {@link OperatorTranslator Translator } for Euphoria {@link Join} operator. */
+public class JoinTranslator<LeftT, RightT, KeyT, OutputT>
+    extends AbstractJoinTranslator<LeftT, RightT, KeyT, OutputT> {
+
+  private abstract static class JoinFn<LeftT, RightT, KeyT, OutputT>
+      extends DoFn<KV<KeyT, CoGbkResult>, KV<KeyT, OutputT>> {
+
+    private final BinaryFunctor<LeftT, RightT, OutputT> joiner;
+    private final TupleTag<LeftT> leftTag;
+    private final TupleTag<RightT> rightTag;
+
+    private final AdaptableCollector<KV<KeyT, CoGbkResult>, KV<KeyT, OutputT>, OutputT>
+        resultsCollector;
+
+    JoinFn(
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        TupleTag<LeftT> leftTag,
+        TupleTag<RightT> rightTag,
+        @Nullable String operatorName,
+        AccumulatorProvider accumulatorProvider) {
+      this.joiner = joiner;
+      this.leftTag = leftTag;
+      this.rightTag = rightTag;
+      this.resultsCollector =
+          new AdaptableCollector<>(
+              accumulatorProvider,
+              operatorName,
+              ((ctx, elem) -> ctx.output(KV.of(ctx.element().getKey(), elem))));
+    }
+
+    @ProcessElement
+    @SuppressWarnings("unused")
+    public final void processElement(@Element KV<KeyT, CoGbkResult> element, ProcessContext ctx) {
+      getCollector().setProcessContext(ctx);
+      doJoin(
+          requireNonNull(element.getValue()).getAll(leftTag),
+          requireNonNull(element.getValue()).getAll(rightTag));
+    }
+
+    abstract void doJoin(Iterable<LeftT> left, Iterable<RightT> right);
+
+    abstract String getFnName();
+
+    BinaryFunctor<LeftT, RightT, OutputT> getJoiner() {
+      return joiner;
+    }
+
+    AdaptableCollector<KV<KeyT, CoGbkResult>, KV<KeyT, OutputT>, OutputT> getCollector() {
+      return resultsCollector;
+    }
+  }
+
+  private static class InnerJoinFn<LeftT, RightT, KeyT, OutputT>
+      extends JoinFn<LeftT, RightT, KeyT, OutputT> {
+
+    InnerJoinFn(
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        TupleTag<LeftT> leftTag,
+        TupleTag<RightT> rightTag,
+        @Nullable String operatorName,
+        AccumulatorProvider accumulatorProvider) {
+      super(joiner, leftTag, rightTag, operatorName, accumulatorProvider);
+    }
+
+    @Override
+    protected void doJoin(Iterable<LeftT> left, Iterable<RightT> right) {
+      for (LeftT leftItem : left) {
+        for (RightT rightItem : right) {
+          getJoiner().apply(leftItem, rightItem, getCollector());
+        }
+      }
+    }
+
+    @Override
+    String getFnName() {
+      return "inner-join";
+    }
+  }
+
+  private static class FullJoinFn<LeftT, RightT, K, OutputT>
+      extends JoinFn<LeftT, RightT, K, OutputT> {
+
+    FullJoinFn(
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        TupleTag<LeftT> leftTag,
+        TupleTag<RightT> rightTag,
+        @Nullable String operatorName,
+        AccumulatorProvider accumulatorProvider) {
+      super(joiner, leftTag, rightTag, operatorName, accumulatorProvider);
+    }
+
+    @Override
+    void doJoin(Iterable<LeftT> left, Iterable<RightT> right) {
+      final boolean leftHasValues = left.iterator().hasNext();
+      final boolean rightHasValues = right.iterator().hasNext();
+      if (leftHasValues && rightHasValues) {
+        for (RightT rightValue : right) {
+          for (LeftT leftValue : left) {
+            getJoiner().apply(leftValue, rightValue, getCollector());
+          }
+        }
+      } else if (leftHasValues) {
+        for (LeftT leftValue : left) {
+          getJoiner().apply(leftValue, null, getCollector());
+        }
+      } else if (rightHasValues) {
+        for (RightT rightValue : right) {
+          getJoiner().apply(null, rightValue, getCollector());
+        }
+      }
+    }
+
+    @Override
+    public String getFnName() {
+      return "full-join";
+    }
+  }
+
+  private static class LeftOuterJoinFn<LeftT, RightT, K, OutputT>
+      extends JoinFn<LeftT, RightT, K, OutputT> {
+
+    LeftOuterJoinFn(
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        TupleTag<LeftT> leftTag,
+        TupleTag<RightT> rightTag,
+        @Nullable String operatorName,
+        AccumulatorProvider accumulatorProvider) {
+      super(joiner, leftTag, rightTag, operatorName, accumulatorProvider);
+    }
+
+    @Override
+    void doJoin(Iterable<LeftT> left, Iterable<RightT> right) {
+      for (LeftT leftValue : left) {
+        if (right.iterator().hasNext()) {
+          for (RightT rightValue : right) {
+            getJoiner().apply(leftValue, rightValue, getCollector());
+          }
+        } else {
+          getJoiner().apply(leftValue, null, getCollector());
+        }
+      }
+    }
+
+    @Override
+    public String getFnName() {
+      return "left-outer-join";
+    }
+  }
+
+  private static class RightOuterJoinFn<LeftT, RightT, K, OutputT>
+      extends JoinFn<LeftT, RightT, K, OutputT> {
+
+    RightOuterJoinFn(
+        BinaryFunctor<LeftT, RightT, OutputT> joiner,
+        TupleTag<LeftT> leftTag,
+        TupleTag<RightT> rightTag,
+        @Nullable String operatorName,
+        AccumulatorProvider accumulatorProvider) {
+      super(joiner, leftTag, rightTag, operatorName, accumulatorProvider);
+    }
+
+    @Override
+    void doJoin(Iterable<LeftT> left, Iterable<RightT> right) {
+      for (RightT rightValue : right) {
+        if (left.iterator().hasNext()) {
+          for (LeftT leftValue : left) {
+            getJoiner().apply(leftValue, rightValue, getCollector());
+          }
+        } else {
+          getJoiner().apply(null, rightValue, getCollector());
+        }
+      }
+    }
+
+    @Override
+    public String getFnName() {
+      return "::right-outer-join";
+    }
+  }
+
+  private static <KeyT, LeftT, RightT, OutputT> JoinFn<LeftT, RightT, KeyT, OutputT> getJoinFn(
+      Join<LeftT, RightT, KeyT, OutputT> operator,
+      TupleTag<LeftT> leftTag,
+      TupleTag<RightT> rightTag,
+      AccumulatorProvider accumulators) {
+    final BinaryFunctor<LeftT, RightT, OutputT> joiner = operator.getJoiner();
+    switch (operator.getType()) {
+      case INNER:
+        return new InnerJoinFn<>(
+            joiner, leftTag, rightTag, operator.getName().orElse(null), accumulators);
+      case LEFT:
+        return new LeftOuterJoinFn<>(
+            joiner, leftTag, rightTag, operator.getName().orElse(null), accumulators);
+      case RIGHT:
+        return new RightOuterJoinFn<>(
+            joiner, leftTag, rightTag, operator.getName().orElse(null), accumulators);
+      case FULL:
+        return new FullJoinFn<>(
+            joiner, leftTag, rightTag, operator.getName().orElse(null), accumulators);
+      default:
+        throw new UnsupportedOperationException(
+            String.format(
+                "Cannot translate Euphoria '%s' operator to Beam transformations."
+                    + " Given join type '%s' is not supported.",
+                Join.class.getSimpleName(), operator.getType()));
+    }
+  }
+
+  @Override
+  PCollection<KV<KeyT, OutputT>> translate(
+      Join<LeftT, RightT, KeyT, OutputT> operator,
+      PCollection<KV<KeyT, LeftT>> left,
+      PCollection<KV<KeyT, RightT>> right) {
+    final AccumulatorProvider accumulators =
+        new LazyAccumulatorProvider(AccumulatorProvider.of(left.getPipeline()));
+    final TupleTag<LeftT> leftTag = new TupleTag<>();
+    final TupleTag<RightT> rightTag = new TupleTag<>();
+    final JoinFn<LeftT, RightT, KeyT, OutputT> joinFn =
+        getJoinFn(operator, leftTag, rightTag, accumulators);
+    return KeyedPCollectionTuple.of(leftTag, left)
+        .and(rightTag, right)
+        .apply("co-group-by-key", CoGroupByKey.create())
+        .apply(joinFn.getFnName(), ParDo.of(joinFn));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java
new file mode 100644
index 00000000000..bfe35b9c184
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+
+/** Instantiate accumulator provider on the first usage. Thus {@link Serializable}. */
+class LazyAccumulatorProvider implements AccumulatorProvider, Serializable {
+
+  private final Factory factory;
+
+  private transient AccumulatorProvider accumulators;
+
+  LazyAccumulatorProvider(Factory factory) {
+    this.factory = requireNonNull(factory);
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return getAccumulatorProvider().getCounter(name);
+  }
+
+  @Override
+  public Counter getCounter(String namespace, String name) {
+    return getAccumulatorProvider().getCounter(namespace, name);
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    return getAccumulatorProvider().getHistogram(name);
+  }
+
+  @Override
+  public Histogram getHistogram(String namespace, String name) {
+    return getAccumulatorProvider().getHistogram(namespace, name);
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    return getAccumulatorProvider().getTimer(name);
+  }
+
+  private AccumulatorProvider getAccumulatorProvider() {
+    if (accumulators == null) {
+      accumulators = factory.create();
+    }
+    return accumulators;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java
new file mode 100644
index 00000000000..e012b0a0a2e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CompositeOperator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+/**
+ * Expand operator to a beam {@link PTransform}.
+ *
+ * @param <InputT> type of input elements
+ * @param <OutputT> type of output elements
+ * @param <OperatorT> type of operator to expand
+ */
+public class OperatorTransform<InputT, OutputT, OperatorT extends Operator<OutputT>>
+    extends PTransform<PCollectionList<InputT>, PCollection<OutputT>> {
+
+  public static <InputT, OutputT, OperatorT extends Operator<OutputT>> Dataset<OutputT> apply(
+      OperatorT operator, List<Dataset<InputT>> inputs) {
+
+    final Optional<OperatorTranslator<InputT, OutputT, OperatorT>> maybeTranslator =
+        TranslatorProvider.of(inputs.get(0).getPipeline()).findTranslator(operator);
+
+    if (maybeTranslator.isPresent()) {
+      final PCollectionList<InputT> inputList =
+          PCollectionList.of(
+              inputs.stream().map(Dataset::getPCollection).collect(Collectors.toList()));
+      final PCollection<OutputT> output =
+          inputList.apply(
+              operator.getName().orElseGet(() -> operator.getClass().getName()),
+              new OperatorTransform<>(operator, maybeTranslator.get()));
+      return Dataset.of(output, operator);
+    }
+
+    if (operator instanceof CompositeOperator) {
+      @SuppressWarnings("unchecked")
+      final CompositeOperator<InputT, OutputT> castedOperator = (CompositeOperator) operator;
+      return Dataset.of(castedOperator.expand(inputs).getPCollection(), operator);
+    }
+
+    throw new IllegalStateException(
+        "Unable to find translator for basic operator ["
+            + operator.getClass()
+            + "] with name ["
+            + operator.getName().orElse(null)
+            + ".");
+  }
+
+  private final OperatorT operator;
+  private final OperatorTranslator<InputT, OutputT, OperatorT> translator;
+
+  private OperatorTransform(
+      OperatorT operator, OperatorTranslator<InputT, OutputT, OperatorT> translator) {
+    this.operator = operator;
+    this.translator = translator;
+  }
+
+  @Override
+  public PCollection<OutputT> expand(PCollectionList<InputT> inputs) {
+    return translator.translate(operator, inputs);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTranslator.java
new file mode 100644
index 00000000000..404a3d20056
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTranslator.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+/**
+ * A functor to translate an operator into a beam execution.
+ *
+ * @param <InputT> the type of input
+ * @param <OutputT> the type of output
+ * @param <OperatorT> the type of the user defined euphoria operator definition
+ */
+public interface OperatorTranslator<InputT, OutputT, OperatorT extends Operator> {
+
+  /**
+   * Translates the given a operator it into a concrete beam transformation.
+   *
+   * @param operator the operator to translate
+   * @param inputs list of inputs
+   * @return a beam transformation
+   */
+  PCollection<OutputT> translate(OperatorT operator, PCollectionList<InputT> inputs);
+
+  /**
+   * Returns true when implementing {@link OperatorTranslator} is able to translate given instance
+   * of an operator, false otherwise.
+   *
+   * <p>This method allow us to have more {@link OperatorTranslator} implementations for one {@link
+   * Operator} in case when some specialized translators are needed.
+   *
+   * @param operator operator to check
+   * @return true if operator can be translated
+   */
+  default boolean canTranslate(OperatorT operator) {
+    return true;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTranslators.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTranslators.java
new file mode 100644
index 00000000000..57c757fca6d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTranslators.java
@@ -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.beam.sdk.extensions.euphoria.core.translate;
+
+import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+/** Utilities related to {@link OperatorTranslator}s. */
+class OperatorTranslators {
+
+  private OperatorTranslators() {}
+
+  static <T> PCollection<T> getSingleInput(PCollectionList<T> inputs) {
+    Preconditions.checkArgument(inputs.size() == 1, "There should be exactly one input.");
+    return inputs.get(0);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java
new file mode 100644
index 00000000000..fdce6c8bf86
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.collector.AdaptableCollector;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.collector.SingleValueCollector;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.TypeDescriptors;
+
+/** Translator for {@code ReduceByKey} operator. */
+public class ReduceByKeyTranslator<InputT, KeyT, ValueT, OutputT>
+    implements OperatorTranslator<
+        InputT, KV<KeyT, OutputT>, ReduceByKey<InputT, KeyT, ValueT, OutputT>> {
+
+  @Override
+  public PCollection<KV<KeyT, OutputT>> translate(
+      ReduceByKey<InputT, KeyT, ValueT, OutputT> operator, PCollectionList<InputT> inputs) {
+
+    // todo Could we even do values sorting in Beam ? And do we want it?
+    checkState(!operator.getValueComparator().isPresent(), "Values sorting is not supported.");
+
+    final UnaryFunction<InputT, KeyT> keyExtractor = operator.getKeyExtractor();
+    final UnaryFunction<InputT, ValueT> valueExtractor = operator.getValueExtractor();
+    final ReduceFunctor<ValueT, OutputT> reducer = operator.getReducer();
+
+    final PCollection<InputT> input =
+        operator
+            .getWindow()
+            .map(window -> OperatorTranslators.getSingleInput(inputs).apply(window))
+            .orElseGet(() -> OperatorTranslators.getSingleInput(inputs));
+
+    // ~ create key & value extractor
+    final MapElements<InputT, KV<KeyT, ValueT>> extractor =
+        MapElements.via(new KeyValueExtractor<>(keyExtractor, valueExtractor));
+
+    final PCollection<KV<KeyT, ValueT>> extracted =
+        input
+            .apply("extract-keys", extractor)
+            .setTypeDescriptor(
+                TypeDescriptors.kvs(
+                    TypeAwares.orObjects(operator.getKeyType()),
+                    TypeAwares.orObjects(operator.getValueType())));
+
+    final AccumulatorProvider accumulators =
+        new LazyAccumulatorProvider(AccumulatorProvider.of(inputs.getPipeline()));
+
+    if (operator.isCombinable()) {
+      // if operator is combinable we can process it in more efficient way
+      final PCollection<KV<KeyT, ValueT>> combined =
+          extracted.apply(
+              "combine",
+              Combine.perKey(asCombiner(reducer, accumulators, operator.getName().orElse(null))));
+      @SuppressWarnings("unchecked")
+      final PCollection<KV<KeyT, OutputT>> casted = (PCollection) combined;
+      return casted;
+    }
+
+    return extracted
+        .apply("group", GroupByKey.create())
+        .setTypeDescriptor(
+            TypeDescriptors.kvs(
+                TypeAwares.orObjects(operator.getKeyType()),
+                TypeDescriptors.iterables(TypeAwares.orObjects(operator.getValueType()))))
+        .apply(
+            "reduce",
+            ParDo.of(new ReduceDoFn<>(reducer, accumulators, operator.getName().orElse(null))))
+        .setTypeDescriptor(
+            operator
+                .getOutputType()
+                .orElseThrow(
+                    () -> new IllegalStateException("Unable to infer output type descriptor.")));
+  }
+
+  @Override
+  public boolean canTranslate(ReduceByKey operator) {
+    // translation of sorted values is not supported yet
+    return !operator.getValueComparator().isPresent();
+  }
+
+  private static <InputT, OutputT> SerializableFunction<Iterable<InputT>, InputT> asCombiner(
+      ReduceFunctor<InputT, OutputT> reducer,
+      AccumulatorProvider accumulatorProvider,
+      @Nullable String operatorName) {
+
+    @SuppressWarnings("unchecked")
+    final ReduceFunctor<InputT, InputT> combiner = (ReduceFunctor<InputT, InputT>) reducer;
+
+    return (Iterable<InputT> input) -> {
+      SingleValueCollector<InputT> collector =
+          new SingleValueCollector<>(accumulatorProvider, operatorName);
+      combiner.apply(StreamSupport.stream(input.spliterator(), false), collector);
+      return collector.get();
+    };
+  }
+
+  /**
+   * Extract key and values from input data set.
+   *
+   * @param <InputT> type of input
+   * @param <KeyT> type of key
+   * @param <ValueT> type of value
+   */
+  private static class KeyValueExtractor<InputT, KeyT, ValueT>
+      extends SimpleFunction<InputT, KV<KeyT, ValueT>> {
+
+    private final UnaryFunction<InputT, KeyT> keyExtractor;
+    private final UnaryFunction<InputT, ValueT> valueExtractor;
+
+    KeyValueExtractor(
+        UnaryFunction<InputT, KeyT> keyExtractor, UnaryFunction<InputT, ValueT> valueExtractor) {
+      this.keyExtractor = keyExtractor;
+      this.valueExtractor = valueExtractor;
+    }
+
+    @Override
+    public KV<KeyT, ValueT> apply(InputT in) {
+      return KV.of(keyExtractor.apply(in), valueExtractor.apply(in));
+    }
+  }
+
+  /**
+   * Perform reduction of given elements.
+   *
+   * @param <KeyT> type of key
+   * @param <ValueT> type of value
+   * @param <OutputT> type of output
+   */
+  private static class ReduceDoFn<KeyT, ValueT, OutputT>
+      extends DoFn<KV<KeyT, Iterable<ValueT>>, KV<KeyT, OutputT>> {
+
+    private final ReduceFunctor<ValueT, OutputT> reducer;
+    private final AdaptableCollector<KV<KeyT, Iterable<ValueT>>, KV<KeyT, OutputT>, OutputT>
+        collector;
+
+    ReduceDoFn(
+        ReduceFunctor<ValueT, OutputT> reducer,
+        AccumulatorProvider accumulators,
+        @Nullable String operatorName) {
+      this.reducer = reducer;
+      this.collector =
+          new AdaptableCollector<>(
+              accumulators,
+              operatorName,
+              (DoFn<KV<KeyT, Iterable<ValueT>>, KV<KeyT, OutputT>>.ProcessContext ctx,
+                  OutputT out) -> ctx.output(KV.of(ctx.element().getKey(), out)));
+    }
+
+    @ProcessElement
+    @SuppressWarnings("unused")
+    public void processElement(ProcessContext ctx) {
+      collector.setProcessContext(ctx);
+      reducer.apply(
+          StreamSupport.stream(requireNonNull(ctx.element().getValue()).spliterator(), false),
+          collector);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SimpleTranslatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SimpleTranslatorProvider.java
new file mode 100644
index 00000000000..6ed1fdf235f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SimpleTranslatorProvider.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import java.util.Collection;
+import java.util.Optional;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+
+/**
+ * Default translation provider that selects first matching translation for the registered operator.
+ */
+public class SimpleTranslatorProvider implements TranslatorProvider {
+
+  /**
+   * Create a new builder for provider.
+   *
+   * @return builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+
+    private final Multimap<Class<? extends Operator>, OperatorTranslator<?, ?, ?>> translators =
+        ArrayListMultimap.create();
+
+    private Builder() {}
+
+    public Builder registerTranslator(
+        Class<? extends Operator> clazz, OperatorTranslator<?, ?, ?> operatorTranslator) {
+      translators.put(clazz, operatorTranslator);
+      return this;
+    }
+
+    public SimpleTranslatorProvider build() {
+      return new SimpleTranslatorProvider(translators);
+    }
+  }
+
+  private final Multimap<Class<? extends Operator>, OperatorTranslator<?, ?, ?>> translators;
+
+  private SimpleTranslatorProvider(
+      Multimap<Class<? extends Operator>, OperatorTranslator<?, ?, ?>> translators) {
+    this.translators = translators;
+  }
+
+  @Override
+  public <InputT, OutputT, OperatorT extends Operator<OutputT>>
+      Optional<OperatorTranslator<InputT, OutputT, OperatorT>> findTranslator(OperatorT operator) {
+    @SuppressWarnings("unchecked")
+    final Collection<OperatorTranslator<InputT, OutputT, OperatorT>> candidates =
+        (Collection) translators.get(operator.getClass());
+    if (!candidates.isEmpty()) {
+      for (OperatorTranslator<InputT, OutputT, OperatorT> candidate : candidates) {
+        if (candidate.canTranslate(operator)) {
+          return Optional.of(candidate);
+        }
+      }
+    }
+    return Optional.empty();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java
new file mode 100644
index 00000000000..2e3121fbf65
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context;
+
+/**
+ * A {@code Context} that holds only single value. There is no window associated with the value, so
+ * the {@code getWindow()} will always throw {@code UnsupportedOperationException}. This context
+ * will free the value as soon as {@code getAndResetValue()} is called.
+ */
+@Audience(Audience.Type.EXECUTOR)
+public class SingleValueContext<T> implements Context, Collector<T> {
+
+  @Nullable private final Context wrap;
+  private T value;
+
+  public SingleValueContext() {
+    this(null);
+  }
+
+  public SingleValueContext(@Nullable Context wrap) {
+    this.wrap = wrap;
+  }
+
+  /**
+   * Replace the stored value with given one.
+   *
+   * @param elem the element to store
+   */
+  @Override
+  public void collect(T elem) {
+    value = elem;
+  }
+
+  @Override
+  public Context asContext() {
+    return this;
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    if (wrap == null) {
+      throw new UnsupportedOperationException("Accumulators not supported in this context");
+    }
+    return wrap.getCounter(name);
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    if (wrap == null) {
+      throw new UnsupportedOperationException("Accumulators not supported in this context");
+    }
+    return wrap.getHistogram(name);
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    if (wrap == null) {
+      throw new UnsupportedOperationException("Accumulators not supported in this context");
+    }
+    return wrap.getTimer(name);
+  }
+
+  /**
+   * Retrieve and reset the stored value to null.
+   *
+   * @return the stored value
+   */
+  public T getAndResetValue() {
+    T ret = value;
+    value = null;
+    return ret;
+  }
+
+  /**
+   * Retrieve value of this context.
+   *
+   * @return value
+   */
+  public T get() {
+    return value;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TranslatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TranslatorProvider.java
new file mode 100644
index 00000000000..0c803f6b8c8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TranslatorProvider.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import java.util.Optional;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+
+/** Simple interface that allows user to define translation of his own. */
+public interface TranslatorProvider {
+
+  static TranslatorProvider of(Pipeline pipeline) {
+    return pipeline.getOptions().as(EuphoriaOptions.class).getTranslatorProvider();
+  }
+
+  /**
+   * Find translation for the given operator. It is possible that no translation exists for the
+   * operator. You can provide your own provider using {@link EuphoriaOptions}.
+   *
+   * @param operator operator to translate
+   * @param <InputT> input type
+   * @param <OutputT> output type
+   * @param <OperatorT> operator type
+   * @return translation if available
+   */
+  <InputT, OutputT, OperatorT extends Operator<OutputT>>
+      Optional<OperatorTranslator<InputT, OutputT, OperatorT>> findTranslator(OperatorT operator);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java
new file mode 100644
index 00000000000..f3eccef826a
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+
+/** Euphoria to Beam translation of {@link Union} operator. */
+class UnionTranslator<InputT> implements OperatorTranslator<InputT, InputT, Union<InputT>> {
+
+  @Override
+  public PCollection<InputT> translate(Union<InputT> operator, PCollectionList<InputT> inputs) {
+    return operator
+        .getName()
+        .map(name -> inputs.apply(name, Flatten.pCollections()))
+        .orElseGet(() -> inputs.apply(Flatten.pCollections()));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java
new file mode 100644
index 00000000000..3c190c1fe91
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate.collector;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * Implementation of {@link Collector} which forwards output elements through {@link
+ * CollectorAdapter} to given {@link DoFn.ProcessContext}. The {@link DoFn.ProcessContext} needs to
+ * be set by {@link AdaptableCollector#setProcessContext(DoFn.ProcessContext)} manually before use.
+ */
+@NotThreadSafe
+@Audience(Audience.Type.EXECUTOR)
+public class AdaptableCollector<InputT, OutputT, ElemT>
+    implements Collector<ElemT>, Context, Serializable {
+
+  private static final String UNSUPPORTED = "Accumulators are supported for named operators only.";
+
+  private final AccumulatorProvider accumulators;
+  private final CollectorAdapter<InputT, OutputT, ElemT> adapter;
+  @Nullable private final String operatorName;
+  private transient DoFn<InputT, OutputT>.ProcessContext context;
+
+  public AdaptableCollector(
+      AccumulatorProvider accumulators,
+      @Nullable String operatorName,
+      CollectorAdapter<InputT, OutputT, ElemT> adapter) {
+    this.accumulators = accumulators;
+    this.operatorName = operatorName;
+    this.adapter = adapter;
+  }
+
+  @Override
+  public void collect(ElemT elem) {
+    adapter.collect(requireNonNull(context), elem);
+  }
+
+  @Override
+  public Context asContext() {
+    return this;
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return accumulators.getCounter(requireNonNull(operatorName, UNSUPPORTED), name);
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    return accumulators.getHistogram(requireNonNull(operatorName, UNSUPPORTED), name);
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    throw new UnsupportedOperationException("Timer not supported. Use histogram instead.");
+  }
+
+  public void setProcessContext(DoFn<InputT, OutputT>.ProcessContext context) {
+    this.context = requireNonNull(context);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/CollectorAdapter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/CollectorAdapter.java
new file mode 100644
index 00000000000..42e3720243e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/CollectorAdapter.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate.collector;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.transforms.DoFn;
+
+/**
+ * User specified adaptation of {@link DoFn.ProcessContext } to Euphoria's {@link Collector}.
+ *
+ * @param <OutputT> type of output
+ * @param <ElemT> type of element
+ */
+public interface CollectorAdapter<InputT, OutputT, ElemT> extends Serializable {
+
+  /**
+   * Collect element. Implementations should use {@link DoFn.ProcessContext#output(Object)} to write
+   * output elements of type {@code OutputT}.
+   *
+   * @param ctx process context
+   * @param elem element
+   */
+  void collect(DoFn<InputT, OutputT>.ProcessContext ctx, ElemT elem);
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java
new file mode 100644
index 00000000000..2ef3755f09e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate.collector;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context;
+
+/** {@code Collector} for combinable functors. */
+public class SingleValueCollector<T> implements Collector<T>, Serializable {
+
+  private static final String UNSUPPORTED = "Accumulators are supported for named operators only.";
+
+  private final AccumulatorProvider accumulators;
+  @Nullable private final String operatorName;
+  private T elem;
+
+  public SingleValueCollector(AccumulatorProvider accumulators, @Nullable String operatorName) {
+    this.accumulators = accumulators;
+    this.operatorName = operatorName;
+  }
+
+  public T get() {
+    return elem;
+  }
+
+  @Override
+  public void collect(T elem) {
+    this.elem = elem;
+  }
+
+  @Override
+  public Context asContext() {
+    // this is not needed, the underlying functor does not have access to this
+    throw new UnsupportedOperationException("Not supported.");
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return accumulators.getCounter(requireNonNull(operatorName, UNSUPPORTED), name);
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    return accumulators.getHistogram(requireNonNull(operatorName, UNSUPPORTED), name);
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    return accumulators.getTimer(name);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/package-info.java
new file mode 100644
index 00000000000..7f710c4baf8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Implementations of {@link org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector}. */
+package org.apache.beam.sdk.extensions.euphoria.core.translate.collector;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/IOUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/IOUtils.java
new file mode 100644
index 00000000000..a7ead68a061
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/IOUtils.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.util;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+/** Util class that helps iterate over methods throwing {@link IOException}. */
+public class IOUtils {
+
+  /**
+   * Accepts consumer for each element. First occurred IOException is thrown after all elements are
+   * iterated. Other IOExceptions are appended as suppressed.
+   *
+   * @param iterable list of elements
+   * @param consumer that performs accept operation
+   * @param <T> type of element
+   * @throws IOException first occurred IOException
+   */
+  public static <T> void forEach(Iterable<T> iterable, IOConsumer<T> consumer) throws IOException {
+    IOException firstException = null;
+    for (T element : iterable) {
+      try {
+        consumer.accept(element);
+      } catch (IOException e) {
+        if (firstException != null) {
+          firstException.addSuppressed(e);
+        } else {
+          firstException = e;
+        }
+      }
+    }
+    if (firstException != null) {
+      throw firstException;
+    }
+  }
+
+  public static <T> void forEach(Stream<T> stream, IOConsumer<T> consumer) throws IOException {
+    forEach(stream::iterator, consumer);
+  }
+
+  /** User defined {@link IOException} throwing consumer. */
+  @FunctionalInterface
+  public interface IOConsumer<T> {
+    void accept(T t) throws IOException;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java
new file mode 100644
index 00000000000..b0903474d74
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.util;
+
+import java.lang.reflect.Constructor;
+import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience;
+
+/**
+ * Util class that helps instantiations of objects throwing {@link RuntimeException}. For core
+ * purposes only. Should not be used in client code.
+ */
+@Audience(Audience.Type.EXECUTOR)
+public class InstanceUtils {
+
+  public static <T> T create(Class<T> cls) {
+    try {
+      Constructor<T> constr = cls.getDeclaredConstructor();
+      constr.setAccessible(true);
+      return constr.newInstance();
+    } catch (Exception e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  public static <T> T create(String className, Class<T> superType) {
+    return create(forName(className, superType));
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <T> Class<? extends T> forName(String className, Class<T> superType) {
+    try {
+      Class<?> cls = Thread.currentThread().getContextClassLoader().loadClass(className);
+      if (superType.isAssignableFrom(cls)) {
+        return (Class<? extends T>) cls;
+      } else {
+        throw new IllegalStateException(className + " is not " + superType);
+      }
+    } catch (ClassNotFoundException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/package-info.java
new file mode 100644
index 00000000000..1a705df1bf5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** A collection of utility classes. */
+package org.apache.beam.sdk.extensions.euphoria.core.util;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java
new file mode 100644
index 00000000000..c73ef137e4a
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.lib;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.OperatorTests;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test suite for {@link Split} library. */
+public class SplitTest {
+
+  @Test
+  public void testBuild() {
+    final String opName = "split";
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+
+    final Split.Output<String> split =
+        Split.named(opName).of(dataset).using((UnaryPredicate<String>) what -> true).output();
+
+    assertTrue(split.positive().getProducer().isPresent());
+    final Filter positive = (Filter) split.positive().getProducer().get();
+    assertNotNull(positive.getPredicate());
+    assertTrue(split.negative().getProducer().isPresent());
+    final Filter negative = (Filter) split.negative().getProducer().get();
+    assertNotNull(negative.getPredicate());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Split.Output<String> split =
+        Split.of(dataset).using((UnaryPredicate<String>) what -> true).output();
+
+    assertTrue(split.positive().getProducer().isPresent());
+    final Filter positive = (Filter) split.positive().getProducer().get();
+    assertTrue(positive.getName().isPresent());
+    assertEquals(Split.DEFAULT_NAME + Split.POSITIVE_FILTER_SUFFIX, positive.getName().get());
+    assertTrue(split.negative().getProducer().isPresent());
+    final Filter negative = (Filter) split.negative().getProducer().get();
+    assertTrue(negative.getName().isPresent());
+    assertEquals(Split.DEFAULT_NAME + Split.NEGATIVE_FILTER_SUFFIX, negative.getName().get());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testBuild_NegatedPredicate() {
+    final Dataset<Integer> dataset = OperatorTests.createMockDataset(TypeDescriptors.integers());
+    final Split.Output<Integer> split =
+        Split.of(dataset).using((UnaryPredicate<Integer>) what -> what % 2 == 0).output();
+
+    assertTrue(split.negative().getProducer().isPresent());
+    final Filter<Integer> oddNumbers = (Filter) split.negative().getProducer().get();
+    assertFalse(oddNumbers.getPredicate().apply(0));
+    assertFalse(oddNumbers.getPredicate().apply(2));
+    assertFalse(oddNumbers.getPredicate().apply(4));
+    assertTrue(oddNumbers.getPredicate().apply(1));
+    assertTrue(oddNumbers.getPredicate().apply(3));
+    assertTrue(oddNumbers.getPredicate().apply(5));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java
new file mode 100644
index 00000000000..3f34d6f946f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test operator CountByKey. */
+public class CountByKeyTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final FixedWindows windowing = FixedWindows.of(org.joda.time.Duration.standardHours(1));
+    final DefaultTrigger trigger = DefaultTrigger.of();
+    final Dataset<KV<String, Long>> counted =
+        CountByKey.named("CountByKey1")
+            .of(dataset)
+            .keyBy(s -> s)
+            .windowBy(windowing)
+            .triggeredBy(trigger)
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.millis(1000))
+            .output();
+    assertTrue(counted.getProducer().isPresent());
+    final CountByKey count = (CountByKey) counted.getProducer().get();
+    assertTrue(count.getName().isPresent());
+    assertEquals("CountByKey1", count.getName().get());
+    assertNotNull(count.getKeyExtractor());
+    assertTrue(count.getWindow().isPresent());
+    final WindowDesc<?> desc = WindowDesc.of((Window<?>) count.getWindow().get());
+    assertEquals(windowing, desc.getWindowFn());
+    assertEquals(trigger, desc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, desc.getAccumulationMode());
+    assertEquals(Duration.millis(1000), desc.getAllowedLateness());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> counted = CountByKey.of(dataset).keyBy(s -> s).output();
+    assertTrue(counted.getProducer().isPresent());
+    final CountByKey count = (CountByKey) counted.getProducer().get();
+    assertFalse(count.getName().isPresent());
+  }
+
+  @Test
+  public void testBuild_Windowing() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> counted =
+        CountByKey.named("CountByKey1")
+            .of(dataset)
+            .keyBy(s -> s)
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .output();
+    assertTrue(counted.getProducer().isPresent());
+    final CountByKey count = (CountByKey) counted.getProducer().get();
+    assertTrue(count.getWindow().isPresent());
+    final WindowDesc<?> desc = WindowDesc.of((Window<?>) count.getWindow().get());
+    assertEquals(FixedWindows.of(org.joda.time.Duration.standardHours(1)), desc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), desc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, desc.getAccumulationMode());
+  }
+
+  @Test
+  public void testWindow_applyIf() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final FixedWindows windowing = FixedWindows.of(org.joda.time.Duration.standardHours(1));
+    final DefaultTrigger trigger = DefaultTrigger.of();
+    final Dataset<KV<String, Long>> counted =
+        CountByKey.named("CountByKey1")
+            .of(dataset)
+            .keyBy(s -> s)
+            .applyIf(true, b -> b.windowBy(windowing).triggeredBy(trigger).discardingFiredPanes())
+            .output();
+    assertTrue(counted.getProducer().isPresent());
+    final CountByKey count = (CountByKey) counted.getProducer().get();
+    assertTrue(count.getWindow().isPresent());
+    final WindowDesc<?> desc = WindowDesc.of((Window<?>) count.getWindow().get());
+    assertEquals(windowing, desc.getWindowFn());
+    assertEquals(trigger, desc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, desc.getAccumulationMode());
+  }
+
+  @Test
+  public void testBuildTypePropagation() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final TypeDescriptor<String> keyType = TypeDescriptors.strings();
+    final Dataset<KV<String, Long>> counted =
+        CountByKey.named("CountByKey1").of(dataset).keyBy(s -> s, keyType).output();
+    assertTrue(counted.getProducer().isPresent());
+    final CountByKey count = (CountByKey) counted.getProducer().get();
+    assertTrue(count.getKeyType().isPresent());
+    assertEquals(count.getKeyType().get(), keyType);
+    assertTrue(count.getOutputType().isPresent());
+    assertEquals(
+        TypeDescriptors.kvs(keyType, TypeDescriptors.longs()), count.getOutputType().get());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java
new file mode 100644
index 00000000000..217881bc36a
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test operator Distinct. */
+public class DistinctTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final FixedWindows windowing = FixedWindows.of(org.joda.time.Duration.standardHours(1));
+    final DefaultTrigger trigger = DefaultTrigger.of();
+    final Dataset<String> uniq =
+        Distinct.named("Distinct1")
+            .of(dataset)
+            .windowBy(windowing)
+            .triggeredBy(trigger)
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.millis(1000))
+            .output();
+    assertTrue(uniq.getProducer().isPresent());
+    final Distinct distinct = (Distinct) uniq.getProducer().get();
+    assertTrue(distinct.getName().isPresent());
+    assertEquals("Distinct1", distinct.getName().get());
+
+    assertTrue(distinct.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) distinct.getWindow().get());
+    assertEquals(windowing, windowDesc.getWindowFn());
+    assertEquals(trigger, windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+    assertEquals(Duration.millis(1000), windowDesc.getAllowedLateness());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> uniq = Distinct.of(dataset).output();
+    assertTrue(uniq.getProducer().isPresent());
+    final Distinct distinct = (Distinct) uniq.getProducer().get();
+    assertFalse(distinct.getName().isPresent());
+  }
+
+  @Test
+  public void testBuild_Windowing() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> uniq =
+        Distinct.of(dataset)
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .output();
+    assertTrue(uniq.getProducer().isPresent());
+    final Distinct distinct = (Distinct) uniq.getProducer().get();
+    assertTrue(distinct.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) distinct.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+  }
+
+  @Test
+  public void testWindow_applyIf() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> uniq =
+        Distinct.of(dataset)
+            .applyIf(
+                true,
+                b ->
+                    b.windowBy(FixedWindows.of(Duration.standardHours(1)))
+                        .triggeredBy(DefaultTrigger.of())
+                        .discardingFiredPanes())
+            .output();
+    assertTrue(uniq.getProducer().isPresent());
+    final Distinct distinct = (Distinct) uniq.getProducer().get();
+    assertTrue(distinct.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) distinct.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java
new file mode 100644
index 00000000000..8083b4b269e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java
@@ -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.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test operator Filter. */
+public class FilterTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> filtered =
+        Filter.named("Filter1").of(dataset).by(s -> !s.equals("")).output();
+    assertTrue(filtered.getProducer().isPresent());
+    final Filter filter = (Filter) filtered.getProducer().get();
+    assertTrue(filter.getName().isPresent());
+    assertEquals("Filter1", filter.getName().get());
+    assertNotNull(filter.getPredicate());
+  }
+
+  @Test
+  public void testBuild_implicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> filtered = Filter.of(dataset).by(s -> !s.equals("")).output();
+    assertTrue(filtered.getProducer().isPresent());
+    final Filter filter = (Filter) filtered.getProducer().get();
+    assertFalse(filter.getName().isPresent());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java
new file mode 100644
index 00000000000..e8a20837463
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test operator FlatMap. */
+public class FlatMapTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> mapped =
+        FlatMap.named("FlatMap1")
+            .of(dataset)
+            .using((String s, Collector<String> c) -> c.collect(s))
+            .output();
+    assertTrue(mapped.getProducer().isPresent());
+    final FlatMap map = (FlatMap) mapped.getProducer().get();
+    assertTrue(map.getName().isPresent());
+    assertEquals("FlatMap1", map.getName().get());
+    assertNotNull(map.getFunctor());
+    assertFalse(map.getEventTimeExtractor().isPresent());
+  }
+
+  @Test
+  public void testBuild_EventTimeExtractor() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<BigDecimal> mapped =
+        FlatMap.named("FlatMap2")
+            .of(dataset)
+            .using((String s, Collector<BigDecimal> c) -> c.collect(null))
+            .eventTimeBy(Long::parseLong) // ~ consuming the original input elements
+            .output();
+    assertTrue(mapped.getProducer().isPresent());
+    final FlatMap map = (FlatMap) mapped.getProducer().get();
+    assertTrue(map.getName().isPresent());
+    assertEquals("FlatMap2", map.getName().get());
+    assertNotNull(map.getFunctor());
+    assertTrue(map.getEventTimeExtractor().isPresent());
+  }
+
+  @Test
+  public void testBuild_WithCounters() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> mapped =
+        FlatMap.named("FlatMap1")
+            .of(dataset)
+            .using(
+                (String s, Collector<String> c) -> {
+                  c.getCounter("my-counter").increment();
+                  c.collect(s);
+                })
+            .output();
+    assertTrue(mapped.getProducer().isPresent());
+    final FlatMap map = (FlatMap) mapped.getProducer().get();
+    assertTrue(map.getName().isPresent());
+    assertEquals("FlatMap1", map.getName().get());
+    assertNotNull(map.getFunctor());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> mapped =
+        FlatMap.of(dataset).using((String s, Collector<String> c) -> c.collect(s)).output();
+    assertTrue(mapped.getProducer().isPresent());
+    final FlatMap map = (FlatMap) mapped.getProducer().get();
+    assertFalse(map.getName().isPresent());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java
new file mode 100644
index 00000000000..6ebea87c246
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Optional;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypePropagationAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Test operator Join. */
+public class JoinTest {
+
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void testBuild() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        Join.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (String l, String r, Collector<String> c) -> {
+                  // no-op
+                })
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertTrue(join.getName().isPresent());
+    assertEquals("Join1", join.getName().get());
+    assertNotNull(join.getLeftKeyExtractor());
+    assertNotNull(join.getRightKeyExtractor());
+    assertFalse(join.getWindow().isPresent());
+    assertEquals(Join.Type.INNER, join.getType());
+  }
+
+  @Test
+  public void testBuild_OutputValues() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+
+    final Dataset<String> joined =
+        Join.named("JoinValues")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (String l, String r, Collector<String> c) -> {
+                  // no-op
+                })
+            .outputValues();
+    assertTrue(joined.getProducer().isPresent());
+    final MapElements mapElements = (MapElements) joined.getProducer().get();
+    assertTrue(mapElements.getName().isPresent());
+    assertEquals("JoinValues::extract-values", mapElements.getName().get());
+  }
+
+  @Test
+  public void testBuild_WithCounters() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        Join.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (String l, String r, Collector<String> c) -> {
+                  c.getCounter("my-counter").increment();
+                  c.collect(l + r);
+                })
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertTrue(join.getName().isPresent());
+    assertEquals("Join1", join.getName().get());
+    assertNotNull(join.getLeftKeyExtractor());
+    assertNotNull(join.getRightKeyExtractor());
+    assertFalse(join.getWindow().isPresent());
+    assertEquals(Join.Type.INNER, join.getType());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        Join.of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (String l, String r, Collector<String> c) -> {
+                  // no-op
+                })
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertFalse(join.getName().isPresent());
+  }
+
+  @Test
+  public void testBuild_LeftJoin() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        LeftJoin.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (String l, Optional<String> r, Collector<String> c) -> {
+                  // no-op
+                })
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertEquals(Join.Type.LEFT, join.getType());
+  }
+
+  @Test
+  public void testBuild_RightJoin() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        RightJoin.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (Optional<String> l, String r, Collector<String> c) -> {
+                  // no-op
+                })
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertEquals(Join.Type.RIGHT, join.getType());
+  }
+
+  @Test
+  public void testBuild_FullJoin() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        FullJoin.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using(
+                (Optional<String> l, Optional<String> r, Collector<String> c) ->
+                    c.collect(l.orElse(null) + r.orElse(null)))
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertEquals(Join.Type.FULL, join.getType());
+  }
+
+  @Test
+  public void testBuild_Windowing() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        Join.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using((String l, String r, Collector<String> c) -> c.collect(l + r))
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(AfterWatermark.pastEndOfWindow())
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.millis(1000))
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertTrue(join.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) join.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(AfterWatermark.pastEndOfWindow(), windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+    assertEquals(Duration.millis(1000), windowDesc.getAllowedLateness());
+  }
+
+  @Test
+  public void testBuild_OptionalWindowing() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<KV<Integer, String>> joined =
+        Join.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length)
+            .using((String l, String r, Collector<String> c) -> c.collect(l + r))
+            .applyIf(
+                true,
+                b ->
+                    b.windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                        .triggeredBy(AfterWatermark.pastEndOfWindow())
+                        .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES))
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    assertTrue(join.getWindow().isPresent());
+    final Window<?> window = (Window) join.getWindow().get();
+    assertEquals(FixedWindows.of(org.joda.time.Duration.standardHours(1)), window.getWindowFn());
+    assertEquals(AfterWatermark.pastEndOfWindow(), WindowDesc.of(window).getTrigger());
+    assertEquals(
+        AccumulationMode.DISCARDING_FIRED_PANES, WindowDesc.of(window).getAccumulationMode());
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testBuildTypePropagation() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final TypeDescriptor<Integer> keyType = TypeDescriptors.integers();
+    final TypeDescriptor<String> outputType = TypeDescriptors.strings();
+    final Dataset<KV<Integer, String>> joined =
+        Join.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length, keyType)
+            .using(
+                (String l, String r, Collector<String> c) -> {
+                  // no-op
+                },
+                outputType)
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    TypePropagationAssert.assertOperatorTypeAwareness(join, keyType, outputType);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testBuild_LeftJoinTypePropagation() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    TypeDescriptor<Integer> keyType = TypeDescriptors.integers();
+    TypeDescriptor<String> outputType = TypeDescriptors.strings();
+    final Dataset<KV<Integer, String>> joined =
+        LeftJoin.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length, keyType)
+            .using(
+                (String l, Optional<String> r, Collector<String> c) -> {
+                  // no-op
+                },
+                outputType)
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    TypePropagationAssert.assertOperatorTypeAwareness(join, keyType, outputType);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testBuild_RightJoinTypePropagation() {
+    final Pipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final TypeDescriptor<Integer> keyType = TypeDescriptors.integers();
+    final TypeDescriptor<String> outputType = TypeDescriptors.strings();
+    final Dataset<KV<Integer, String>> joined =
+        RightJoin.named("Join1")
+            .of(left, right)
+            .by(String::length, String::length, keyType)
+            .using(
+                (Optional<String> l, String r, Collector<String> c) -> {
+                  // no-op
+                },
+                outputType)
+            .output();
+    assertTrue(joined.getProducer().isPresent());
+    final Join join = (Join) joined.getProducer().get();
+    TypePropagationAssert.assertOperatorTypeAwareness(join, keyType, outputType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java
new file mode 100644
index 00000000000..b24aad66ee8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypePropagationAssert;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test operator MapElement. */
+public class MapElementsTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> mapped = MapElements.named("Map1").of(dataset).using(s -> s).output();
+    assertTrue(mapped.getProducer().isPresent());
+    final MapElements map = (MapElements) mapped.getProducer().get();
+    assertTrue(map.getName().isPresent());
+    assertEquals("Map1", map.getName().get());
+    assertNotNull(map.getMapper());
+  }
+
+  @Test
+  public void testBuild_WithCounters() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> mapped =
+        MapElements.named("Map1")
+            .of(dataset)
+            .using(
+                (input, context) -> {
+                  // use simple counter
+                  context.getCounter("my-counter").increment();
+
+                  return input.toLowerCase();
+                })
+            .output();
+
+    assertTrue(mapped.getProducer().isPresent());
+    final MapElements map = (MapElements) mapped.getProducer().get();
+    assertTrue(map.getName().isPresent());
+    assertEquals("Map1", map.getName().get());
+    assertNotNull(map.getMapper());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<String> mapped = MapElements.of(dataset).using(s -> s).output();
+    assertTrue(mapped.getProducer().isPresent());
+    final MapElements map = (MapElements) mapped.getProducer().get();
+    assertFalse(map.getName().isPresent());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testTypePropagation() {
+    final Dataset<Integer> input = OperatorTests.createMockDataset(TypeDescriptors.integers());
+    final TypeDescriptor<String> outputType = TypeDescriptors.strings();
+    final Dataset<String> mapped =
+        MapElements.named("Int2Str").of(input).using(String::valueOf, outputType).output();
+    assertTrue(mapped.getProducer().isPresent());
+    final MapElements map = (MapElements) mapped.getProducer().get();
+    TypePropagationAssert.assertOperatorTypeAwareness(map, outputType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OperatorTests.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OperatorTests.java
new file mode 100644
index 00000000000..a2a6fae3d19
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OperatorTests.java
@@ -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.beam.sdk.extensions.euphoria.core.client.operator;
+
+import java.util.Optional;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwares;
+import org.apache.beam.sdk.extensions.euphoria.core.coder.KryoCoder;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.EuphoriaOptions;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.OperatorTranslator;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.TranslatorProvider;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** Utility class for easier creating input data sets for operator testing. */
+public class OperatorTests {
+
+  private static class PrimitiveOutputTranslatorProvider implements TranslatorProvider {
+
+    @Override
+    public <InputT, OutputT, OperatorT extends Operator<OutputT>>
+        Optional<OperatorTranslator<InputT, OutputT, OperatorT>> findTranslator(
+            OperatorT operator) {
+      return Optional.of(
+          (op, inputs) ->
+              PCollection.<OutputT>createPrimitiveOutputInternal(
+                      inputs.getPipeline(),
+                      inputs.get(0).getWindowingStrategy(),
+                      inputs.get(0).isBounded(),
+                      null)
+                  .setTypeDescriptor(TypeAwares.orObjects(operator.getOutputType())));
+    }
+  }
+
+  public static TestPipeline createTestPipeline() {
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    pipelineOptions
+        .as(EuphoriaOptions.class)
+        .setTranslatorProvider(new PrimitiveOutputTranslatorProvider());
+    final TestPipeline testPipeline = TestPipeline.fromOptions(pipelineOptions);
+    testPipeline
+        .getCoderRegistry()
+        .registerCoderForClass(Object.class, KryoCoder.withoutClassRegistration());
+    return testPipeline;
+  }
+
+  public static <T> Dataset<T> createMockDataset(TypeDescriptor<T> typeDescriptor) {
+    return createMockDataset(createTestPipeline(), typeDescriptor);
+  }
+
+  public static <T> Dataset<T> createMockDataset(
+      Pipeline pipeline, TypeDescriptor<T> typeDescriptor) {
+    return Dataset.of(pipeline.apply(Create.empty(typeDescriptor)));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java
new file mode 100644
index 00000000000..52764139f97
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.util.stream.StreamSupport;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypePropagationAssert;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test operator ReduceByKey. */
+public class ReduceByKeyTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final FixedWindows windowing = FixedWindows.of(org.joda.time.Duration.standardHours(1));
+    final DefaultTrigger trigger = DefaultTrigger.of();
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.named("ReduceByKey1")
+            .of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .combineBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .windowBy(windowing)
+            .triggeredBy(trigger)
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.standardSeconds(1000))
+            .output();
+
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertTrue(reduce.getName().isPresent());
+    assertEquals("ReduceByKey1", reduce.getName().get());
+    assertNotNull(reduce.getKeyExtractor());
+    assertNotNull(reduce.getValueExtractor());
+    assertNotNull(reduce.getReducer());
+
+    assertTrue(reduce.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) reduce.getWindow().get());
+    assertEquals(windowing, windowDesc.getWindowFn());
+    assertEquals(trigger, windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+    assertEquals(Duration.standardSeconds(1000), windowDesc.getAllowedLateness());
+  }
+
+  @Test
+  public void testBuild_OutputValues() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Long> reduced =
+        ReduceByKey.named("ReduceByKeyValues")
+            .of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .reduceBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .outputValues();
+
+    assertTrue(reduced.getProducer().isPresent());
+    final MapElements reduce = (MapElements) reduced.getProducer().get();
+    assertTrue(reduce.getName().isPresent());
+    assertEquals("ReduceByKeyValues::extract-values", reduce.getName().get());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .combineBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertFalse(reduce.getName().isPresent());
+  }
+
+  @Test
+  public void testBuild_ReduceBy() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .reduceBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertNotNull(reduce.getReducer());
+  }
+
+  @Test
+  public void testBuild_Windowing() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .combineBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .windowBy(FixedWindows.of(Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .output();
+
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+
+    assertTrue(reduce.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final Window<? extends BoundedWindow> window = (Window) reduce.getWindow().get();
+    assertEquals(FixedWindows.of(org.joda.time.Duration.standardHours(1)), window.getWindowFn());
+    assertEquals(DefaultTrigger.of(), WindowDesc.of(window).getTrigger());
+    assertSame(
+        AccumulationMode.DISCARDING_FIRED_PANES, WindowDesc.of(window).getAccumulationMode());
+    assertFalse(reduce.getValueComparator().isPresent());
+  }
+
+  @Test
+  public void testBuild_sortedValues() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .reduceBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .withSortedValues(Long::compare)
+            .windowBy(FixedWindows.of(Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertTrue(reduce.getValueComparator().isPresent());
+  }
+
+  @Test
+  public void testBuild_sortedValuesWithNoWindowing() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .reduceBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .withSortedValues(Long::compare)
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertTrue(reduce.getValueComparator().isPresent());
+  }
+
+  @Test
+  public void testWindow_applyIf() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .reduceBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .applyIf(
+                true,
+                b ->
+                    b.windowBy(FixedWindows.of(Duration.standardHours(1)))
+                        .triggeredBy(DefaultTrigger.of())
+                        .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES))
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertTrue(reduce.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final Window<? extends BoundedWindow> window = (Window) reduce.getWindow().get();
+    assertEquals(FixedWindows.of(org.joda.time.Duration.standardHours(1)), window.getWindowFn());
+    assertEquals(DefaultTrigger.of(), WindowDesc.of(window).getTrigger());
+    assertSame(
+        AccumulationMode.DISCARDING_FIRED_PANES, WindowDesc.of(window).getAccumulationMode());
+  }
+
+  @Test
+  public void testWindow_applyIfNot() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .reduceBy(n -> StreamSupport.stream(n.spliterator(), false).mapToLong(Long::new).sum())
+            .applyIf(
+                false,
+                b ->
+                    b.windowBy(FixedWindows.of(Duration.standardHours(1)))
+                        .triggeredBy(DefaultTrigger.of())
+                        .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES))
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    assertFalse(reduce.getWindow().isPresent());
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testTypeHints_typePropagation() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final TypeDescriptor<String> keyType = TypeDescriptors.strings();
+    final TypeDescriptor<Long> valueType = TypeDescriptors.longs();
+    final TypeDescriptor<Long> outputType = TypeDescriptors.longs();
+    final Dataset<KV<String, Long>> reduced =
+        ReduceByKey.of(dataset)
+            .keyBy(s -> s, keyType)
+            .valueBy(s -> 1L, valueType)
+            .combineBy(n -> n.mapToLong(l -> l).sum(), outputType)
+            .output();
+    assertTrue(reduced.getProducer().isPresent());
+    final ReduceByKey reduce = (ReduceByKey) reduced.getProducer().get();
+    TypePropagationAssert.assertOperatorTypeAwareness(reduce, keyType, valueType, outputType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java
new file mode 100644
index 00000000000..8e2e7626ea1
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.stream.Stream;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.SingleValueContext;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test behavior of operator {@code ReduceWindow}. */
+public class ReduceWindowTest {
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testSimpleBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Long> output =
+        ReduceWindow.of(dataset).valueBy(e -> "").reduceBy(e -> 1L).output();
+    assertTrue(output.getProducer().isPresent());
+    final ReduceWindow rw = (ReduceWindow) output.getProducer().get();
+    assertEquals(1L, (long) collectSingle(rw.getReducer(), Stream.of("blah")));
+    assertEquals("", rw.getValueExtractor().apply("blah"));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testSimpleBuildWithoutValue() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Long> output =
+        ReduceWindow.of(dataset)
+            .reduceBy(e -> 1L)
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.millis(1000))
+            .output();
+
+    assertTrue(output.getProducer().isPresent());
+    final ReduceWindow rw = (ReduceWindow) output.getProducer().get();
+
+    assertEquals(1L, (long) collectSingle(rw.getReducer(), Stream.of("blah")));
+    assertEquals("blah", rw.getValueExtractor().apply("blah"));
+
+    assertTrue(rw.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) rw.getWindow().get());
+    assertNotNull(windowDesc);
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+    assertEquals(Duration.millis(1000), windowDesc.getAllowedLateness());
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testSimpleBuildWithValueSorted() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Long> output =
+        ReduceWindow.of(dataset)
+            .reduceBy(e -> 1L)
+            .withSortedValues(String::compareTo)
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .output();
+    assertTrue(output.getProducer().isPresent());
+    final ReduceWindow rw = (ReduceWindow) output.getProducer().get();
+    assertTrue(rw.getValueComparator().isPresent());
+  }
+
+  @Test
+  public void testWindow_applyIf() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Long> output =
+        ReduceWindow.of(dataset)
+            .reduceBy(e -> 1L)
+            .withSortedValues(String::compareTo)
+            .applyIf(
+                true,
+                b ->
+                    b.windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                        .triggeredBy(DefaultTrigger.of())
+                        .discardingFiredPanes())
+            .output();
+    assertTrue(output.getProducer().isPresent());
+    final ReduceWindow rw = (ReduceWindow) output.getProducer().get();
+    assertTrue(rw.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) rw.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+  }
+
+  private <InputT, OutputT> OutputT collectSingle(
+      ReduceFunctor<InputT, OutputT> fn, Stream<InputT> values) {
+    final SingleValueContext<OutputT> context = new SingleValueContext<>();
+    fn.apply(values, context);
+    return context.getAndResetValue();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java
new file mode 100644
index 00000000000..81801e793e5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test behavior of operator {@code SumByKey}. */
+public class SumByKeyTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> counted =
+        SumByKey.named("SumByKey1").of(dataset).keyBy(s -> s).output();
+    assertTrue(counted.getProducer().isPresent());
+    final SumByKey sum = (SumByKey) counted.getProducer().get();
+    assertTrue(sum.getName().isPresent());
+    assertEquals("SumByKey1", sum.getName().get());
+    assertNotNull(sum.getKeyExtractor());
+    assertFalse(sum.getWindow().isPresent());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> counted = SumByKey.of(dataset).keyBy(s -> s).output();
+    assertTrue(counted.getProducer().isPresent());
+    final SumByKey sum = (SumByKey) counted.getProducer().get();
+    assertFalse(sum.getName().isPresent());
+  }
+
+  @Test
+  public void testBuild_Windowing() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> counted =
+        SumByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.millis(1000))
+            .output();
+    assertTrue(counted.getProducer().isPresent());
+    final SumByKey sum = (SumByKey) counted.getProducer().get();
+    assertTrue(sum.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) sum.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+    assertEquals(Duration.millis(1000), windowDesc.getAllowedLateness());
+  }
+
+  @Test
+  public void testWindow_applyIf() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<KV<String, Long>> counted =
+        SumByKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .applyIf(
+                true,
+                b ->
+                    b.windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                        .triggeredBy(DefaultTrigger.of())
+                        .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES))
+            .output();
+    assertTrue(counted.getProducer().isPresent());
+    final SumByKey sum = (SumByKey) counted.getProducer().get();
+    assertTrue(sum.getWindow().isPresent());
+    final Window<?> window = (Window) sum.getWindow().get();
+    assertEquals(FixedWindows.of(org.joda.time.Duration.standardHours(1)), window.getWindowFn());
+    assertEquals(DefaultTrigger.of(), WindowDesc.of(window).getTrigger());
+    assertEquals(
+        AccumulationMode.DISCARDING_FIRED_PANES, WindowDesc.of(window).getAccumulationMode());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java
new file mode 100644
index 00000000000..78241f880f0
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowDesc;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test behavior of operator {@code TopPerKey}. */
+public class TopPerKeyTest {
+
+  @Test
+  public void testBuild() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final FixedWindows windowing = FixedWindows.of(org.joda.time.Duration.standardHours(1));
+    final DefaultTrigger trigger = DefaultTrigger.of();
+    final Dataset<Triple<String, Long, Long>> result =
+        TopPerKey.named("TopPerKey1")
+            .of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .scoreBy(s -> 1L)
+            .windowBy(windowing)
+            .triggeredBy(trigger)
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.millis(1000))
+            .output();
+    assertTrue(result.getProducer().isPresent());
+    final TopPerKey tpk = (TopPerKey) result.getProducer().get();
+    assertTrue(tpk.getName().isPresent());
+    assertEquals("TopPerKey1", tpk.getName().get());
+    assertNotNull(tpk.getKeyExtractor());
+    assertNotNull(tpk.getValueExtractor());
+    assertNotNull(tpk.getScoreExtractor());
+
+    assertTrue(tpk.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) tpk.getWindow().get());
+    assertEquals(windowing, windowDesc.getWindowFn());
+    assertEquals(trigger, windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+    assertEquals(Duration.millis(1000), windowDesc.getAllowedLateness());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Triple<String, Long, Long>> result =
+        TopPerKey.of(dataset).keyBy(s -> s).valueBy(s -> 1L).scoreBy(s -> 1L).output();
+    assertTrue(result.getProducer().isPresent());
+    final TopPerKey tpk = (TopPerKey) result.getProducer().get();
+    assertFalse(tpk.getName().isPresent());
+  }
+
+  @Test
+  public void testBuild_Windowing() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Triple<String, Long, Long>> result =
+        TopPerKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .scoreBy(s -> 1L)
+            .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+            .triggeredBy(DefaultTrigger.of())
+            .accumulationMode(AccumulationMode.DISCARDING_FIRED_PANES)
+            .output();
+    assertTrue(result.getProducer().isPresent());
+    final TopPerKey tpk = (TopPerKey) result.getProducer().get();
+    assertTrue(tpk.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) tpk.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+    assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, windowDesc.getAccumulationMode());
+  }
+
+  @Test
+  public void testWindow_applyIf() {
+    final Dataset<String> dataset = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    final Dataset<Triple<String, Long, Long>> result =
+        TopPerKey.of(dataset)
+            .keyBy(s -> s)
+            .valueBy(s -> 1L)
+            .scoreBy(s -> 1L)
+            .applyIf(
+                true,
+                b ->
+                    b.windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                        .triggeredBy(DefaultTrigger.of())
+                        .accumulatingFiredPanes())
+            .output();
+    assertTrue(result.getProducer().isPresent());
+    final TopPerKey tpk = (TopPerKey) result.getProducer().get();
+    assertTrue(tpk.getWindow().isPresent());
+    @SuppressWarnings("unchecked")
+    final WindowDesc<?> windowDesc = WindowDesc.of((Window) tpk.getWindow().get());
+    assertEquals(
+        FixedWindows.of(org.joda.time.Duration.standardHours(1)), windowDesc.getWindowFn());
+    assertEquals(DefaultTrigger.of(), windowDesc.getTrigger());
+    assertEquals(AccumulationMode.ACCUMULATING_FIRED_PANES, windowDesc.getAccumulationMode());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java
new file mode 100644
index 00000000000..9e95ae6ac12
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.operator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test behavior of operator {@code Union}. */
+public class UnionTest {
+
+  @Test
+  public void testBuild() {
+    final TestPipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+
+    final Dataset<String> unioned = Union.named("Union1").of(left, right).output();
+
+    assertTrue(unioned.getProducer().isPresent());
+    final Union union = (Union) unioned.getProducer().get();
+    assertTrue(union.getName().isPresent());
+    assertEquals("Union1", union.getName().get());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testBuild_OneDataSet() {
+    final Dataset<String> first = OperatorTests.createMockDataset(TypeDescriptors.strings());
+    Union.named("Union1").of(first).output();
+  }
+
+  @Test
+  public void testBuild_ThreeDataSet() {
+    final TestPipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> first =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> second =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> third =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+
+    final Dataset<String> unioned = Union.named("Union1").of(first, second, third).output();
+
+    assertTrue(unioned.getProducer().isPresent());
+    final Union union = (Union) unioned.getProducer().get();
+    assertTrue(union.getName().isPresent());
+    assertEquals("Union1", union.getName().get());
+  }
+
+  @Test
+  public void testBuild_ImplicitName() {
+    final TestPipeline pipeline = OperatorTests.createTestPipeline();
+    final Dataset<String> left =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> right =
+        OperatorTests.createMockDataset(pipeline, TypeDescriptors.strings());
+    final Dataset<String> unioned = Union.of(left, right).output();
+    assertTrue(unioned.getProducer().isPresent());
+    final Union union = (Union) unioned.getProducer().get();
+    assertFalse(union.getName().isPresent());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java
new file mode 100644
index 00000000000..60362b9d435
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.client.type;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.lang.reflect.TypeVariable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.Operator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.base.ShuffleOperator;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** Bunch of methods to assert type descriptors in operators. */
+public class TypePropagationAssert {
+
+  public static <KeyT, ValueT, OutputT> void assertOperatorTypeAwareness(
+      Operator<OutputT> operator,
+      @Nullable TypeDescriptor<KeyT> keyType,
+      @Nullable TypeDescriptor<ValueT> valueType,
+      TypeDescriptor<OutputT> outputType) {
+    if (keyType != null || operator instanceof TypeAware.Key) {
+      @SuppressWarnings("unchecked")
+      final TypeAware.Key<KeyT> keyAware = (TypeAware.Key) operator;
+      assertTrue(keyAware.getKeyType().isPresent());
+      assertEquals(keyType, keyAware.getKeyType().get());
+      assertTrue(operator.getOutputType().isPresent());
+      @SuppressWarnings("unchecked")
+      final TypeDescriptor<KV<KeyT, OutputT>> kvOutputType =
+          (TypeDescriptor) operator.getOutputType().get();
+      final TypeVariable<Class<KV>>[] kvParameters = KV.class.getTypeParameters();
+      final TypeDescriptor<?> firstType = kvOutputType.resolveType(kvParameters[0]);
+      final TypeDescriptor<?> secondType = kvOutputType.resolveType(kvParameters[1]);
+      assertEquals(keyType, firstType);
+      assertEquals(outputType, secondType);
+    } else {
+      // assert output of non keyed operator
+      assertEquals(outputType, operator.getOutputType().get());
+    }
+    if (valueType != null || operator instanceof TypeAware.Value) {
+      @SuppressWarnings("unchecked")
+      final TypeAware.Value<KeyT> valueAware = (TypeAware.Value) operator;
+      assertTrue(valueAware.getValueType().isPresent());
+      assertEquals(valueType, valueAware.getValueType().get());
+    }
+  }
+
+  public static <OutputT> void assertOperatorTypeAwareness(
+      Operator<OutputT> operator, TypeDescriptor<OutputT> outputType) {
+    assertOperatorTypeAwareness(operator, null, null, outputType);
+  }
+
+  public static <KeyT, OutputT> void assertOperatorTypeAwareness(
+      ShuffleOperator<?, KeyT, OutputT> operator,
+      TypeDescriptor<KeyT> keyType,
+      TypeDescriptor<OutputT> outputType) {
+    assertOperatorTypeAwareness(operator, keyType, null, outputType);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java
new file mode 100644
index 00000000000..4debb09a505
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java
@@ -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.beam.sdk.extensions.euphoria.core.client.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.Stream;
+import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils;
+import org.junit.Test;
+
+/** Test behavior of IOUtils. */
+public class IOUtilsTest {
+
+  @Test(expected = IOException.class)
+  public void testOneIOException() throws IOException {
+    IOUtils.forEach(
+        Arrays.asList(1, 2, 3),
+        (i) -> {
+          if (i == 2) {
+            throw new IOException("Number: " + i);
+          }
+        });
+  }
+
+  @Test
+  public void testSuppressedIOException() throws IOException {
+    try {
+      IOUtils.forEach(
+          Arrays.asList(1, 2, 3),
+          (i) -> {
+            throw new IOException("Number: " + i);
+          });
+    } catch (Exception e) {
+      assertEquals(2, e.getSuppressed().length); // two supressed exceptions and one thrown
+      assertTrue(e instanceof IOException);
+      assertEquals("Number: 1", e.getMessage());
+    }
+  }
+
+  @Test(expected = IOException.class)
+  public void testStreamIOException() throws IOException {
+
+    IOUtils.forEach(
+        Stream.of(1, 2, 3),
+        (i) -> {
+          if (i == 2) {
+            throw new IOException("Number: " + i);
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/IdentifiedRegistrarTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/IdentifiedRegistrarTest.java
new file mode 100644
index 00000000000..3416d17929c
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/IdentifiedRegistrarTest.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import org.junit.Test;
+
+/** Unit tests of {@link IdentifiedRegistrar}. */
+public class IdentifiedRegistrarTest {
+
+  @Test
+  public void testSerializationRetainsId() throws IOException, ClassNotFoundException {
+    IdentifiedRegistrar registrarWithIdOriginal = IdentifiedRegistrar.of((k) -> {});
+
+    ByteArrayOutputStream outStr = new ByteArrayOutputStream();
+    ObjectOutputStream oss = new ObjectOutputStream(outStr);
+
+    oss.writeObject(registrarWithIdOriginal);
+    oss.flush();
+    oss.close();
+
+    ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(outStr.toByteArray()));
+    IdentifiedRegistrar registrarDeserialized = (IdentifiedRegistrar) ois.readObject();
+
+    assertEquals(registrarWithIdOriginal.getId(), registrarDeserialized.getId());
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoCoderTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoCoderTest.java
new file mode 100644
index 00000000000..8e95d59b527
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoCoderTest.java
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Test targeted at {@link KryoCoder}. */
+public class KryoCoderTest {
+
+  @Test
+  public void testBasicCoding() throws IOException {
+
+    KryoRegistrar registrar = (k) -> k.register(ClassToBeEncoded.class);
+
+    KryoCoder<ClassToBeEncoded> coder = KryoCoder.of(IdentifiedRegistrar.of(registrar));
+    assertEncoding(coder);
+  }
+
+  @Test(expected = CoderException.class)
+  public void testWrongRegistrarCoding() throws IOException {
+
+    KryoRegistrar registrar =
+        (k) -> {
+          // No-op
+        };
+
+    KryoCoder<ClassToBeEncoded> coder = KryoCoder.of(IdentifiedRegistrar.of(registrar));
+    assertEncoding(coder);
+  }
+
+  @Test(expected = CoderException.class)
+  public void testWrongRegistrarDecoding() throws IOException {
+
+    KryoRegistrar registrarCoding = (k) -> k.register(ClassToBeEncoded.class);
+    KryoRegistrar registrarDecoding =
+        (k) -> {
+          // No-op
+        };
+
+    KryoCoder<ClassToBeEncoded> coderToEncode =
+        KryoCoder.of(IdentifiedRegistrar.of(registrarCoding));
+    KryoCoder<ClassToBeEncoded> coderToDecode =
+        KryoCoder.of(IdentifiedRegistrar.of(registrarDecoding));
+
+    assertEncoding(coderToEncode, coderToDecode);
+  }
+
+  @Test
+  public void testCodingOfTwoClassesInSerial() throws IOException {
+    KryoRegistrar registrar =
+        (k) -> {
+          k.register(ClassToBeEncoded.class);
+          k.register(TestClass.class);
+        };
+
+    IdentifiedRegistrar registrarWithId = IdentifiedRegistrar.of(registrar);
+    KryoCoder<ClassToBeEncoded> coder = KryoCoder.of(registrarWithId);
+    KryoCoder<TestClass> secondCoder = KryoCoder.of(registrarWithId);
+
+    ClassToBeEncoded originalValue = new ClassToBeEncoded("XyZ", 42, Double.NaN);
+    TestClass secondOriginalValue = new TestClass("just a parameter");
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+
+    coder.encode(originalValue, outputStream);
+    secondCoder.encode(secondOriginalValue, outputStream);
+
+    byte[] buf = outputStream.toByteArray();
+    ByteArrayInputStream inputStream = new ByteArrayInputStream(buf);
+
+    ClassToBeEncoded decodedValue = coder.decode(inputStream);
+    TestClass secondDecodedValue = secondCoder.decode(inputStream);
+
+    Assert.assertNotNull(decodedValue);
+    Assert.assertEquals(originalValue, decodedValue);
+
+    Assert.assertNotNull(secondDecodedValue);
+    Assert.assertNotNull(secondDecodedValue.param);
+    Assert.assertEquals("just a parameter", secondDecodedValue.param);
+  }
+
+  /** Test whenever the {@link KryoCoder} is serializable. */
+  @Test
+  public void testCoderSerialization() throws IOException, ClassNotFoundException {
+    KryoRegistrar registrar = (k) -> k.register(ClassToBeEncoded.class);
+
+    KryoCoder<ClassToBeEncoded> coder = KryoCoder.of(IdentifiedRegistrar.of(registrar));
+    ByteArrayOutputStream outStr = new ByteArrayOutputStream();
+    ObjectOutputStream oss = new ObjectOutputStream(outStr);
+
+    oss.writeObject(coder);
+    oss.flush();
+    oss.close();
+
+    ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(outStr.toByteArray()));
+    @SuppressWarnings("unchecked")
+    KryoCoder<ClassToBeEncoded> coderDeserialized = (KryoCoder<ClassToBeEncoded>) ois.readObject();
+
+    assertEncoding(coder, coderDeserialized);
+  }
+
+  @Test
+  public void testCodingWithKvCoderKeyIsKryoCoder() throws IOException {
+    KryoRegistrar registrar = (k) -> k.register(TestClass.class);
+
+    final ListCoder<Void> listCoder = ListCoder.of(VoidCoder.of());
+    final KvCoder<TestClass, List<Void>> kvCoder =
+        KvCoder.of(KryoCoder.of(IdentifiedRegistrar.of(registrar)), listCoder);
+
+    List<Void> inputValue = new ArrayList<>();
+    inputValue.add(null);
+    inputValue.add(null);
+
+    final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+
+    TestClass inputKey = new TestClass("something");
+    kvCoder.encode(KV.of(inputKey, inputValue), byteArrayOutputStream);
+
+    final KV<TestClass, List<Void>> decoded =
+        kvCoder.decode(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
+
+    Assert.assertNotNull(decoded);
+    Assert.assertNotNull(decoded.getKey());
+    Assert.assertEquals(inputKey, decoded.getKey());
+
+    Assert.assertNotNull(decoded.getValue());
+    Assert.assertEquals(inputValue, decoded.getValue());
+  }
+
+  @Test
+  public void testCodingWithKvCoderValueIsKryoCoder() throws IOException {
+    KryoRegistrar registrar = (k) -> k.register(TestClass.class);
+
+    final KvCoder<String, TestClass> kvCoder =
+        KvCoder.of(StringUtf8Coder.of(), KryoCoder.of(IdentifiedRegistrar.of(registrar)));
+
+    final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+
+    String inputKey = "key";
+    TestClass inputValue = new TestClass("something");
+    kvCoder.encode(KV.of(inputKey, inputValue), byteArrayOutputStream);
+
+    final KV<String, TestClass> decoded =
+        kvCoder.decode(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
+
+    Assert.assertNotNull(decoded);
+    Assert.assertNotNull(decoded.getKey());
+    Assert.assertEquals(inputKey, decoded.getKey());
+
+    Assert.assertNotNull(decoded.getValue());
+    Assert.assertEquals(inputValue, decoded.getValue());
+  }
+
+  @Test
+  public void testCodingWithKvCoderClassToBeEncoded() throws IOException {
+    KryoRegistrar registrar =
+        (k) -> {
+          k.register(TestClass.class);
+          k.register(ClassToBeEncoded.class);
+        };
+
+    final ListCoder<Void> listCoder = ListCoder.of(VoidCoder.of());
+    final KvCoder<ClassToBeEncoded, List<Void>> kvCoder =
+        KvCoder.of(KryoCoder.of(IdentifiedRegistrar.of(registrar)), listCoder);
+    List<Void> inputValue = new ArrayList<>();
+    inputValue.add(null);
+    inputValue.add(null);
+
+    final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+
+    ClassToBeEncoded inputKey = new ClassToBeEncoded("something", 1, 0.2);
+    kvCoder.encode(KV.of(inputKey, inputValue), byteArrayOutputStream);
+
+    final KV<ClassToBeEncoded, List<Void>> decoded =
+        kvCoder.decode(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
+
+    Assert.assertNotNull(decoded);
+    Assert.assertNotNull(decoded.getKey());
+    Assert.assertEquals(inputKey, decoded.getKey());
+
+    Assert.assertNotNull(decoded.getValue());
+    Assert.assertEquals(inputValue, decoded.getValue());
+  }
+
+  private void assertEncoding(KryoCoder<ClassToBeEncoded> coder) throws IOException {
+    assertEncoding(coder, coder);
+  }
+
+  private void assertEncoding(
+      KryoCoder<ClassToBeEncoded> coderToEncode, KryoCoder<ClassToBeEncoded> coderToDecode)
+      throws IOException {
+
+    ClassToBeEncoded originalValue = new ClassToBeEncoded("XyZ", 42, Double.NaN);
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+
+    coderToEncode.encode(originalValue, outputStream);
+
+    byte[] buf = outputStream.toByteArray();
+    ByteArrayInputStream inputStream = new ByteArrayInputStream(buf);
+
+    ClassToBeEncoded decodedValue = coderToDecode.decode(inputStream);
+
+    Assert.assertNotNull(decodedValue);
+    Assert.assertEquals(originalValue, decodedValue);
+  }
+
+  private static class ClassToBeEncoded {
+
+    private String firstField;
+    private Integer secondField;
+    private Double thirdField;
+
+    ClassToBeEncoded(String firstField, Integer secondField, Double thirdField) {
+      this.firstField = firstField;
+      this.secondField = secondField;
+      this.thirdField = thirdField;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      ClassToBeEncoded that = (ClassToBeEncoded) o;
+      return Objects.equals(firstField, that.firstField)
+          && Objects.equals(secondField, that.secondField)
+          && Objects.equals(thirdField, that.thirdField);
+    }
+
+    @Override
+    public int hashCode() {
+
+      return Objects.hash(firstField, secondField, thirdField);
+    }
+  }
+
+  static class TestClass {
+
+    String param;
+
+    TestClass(String param) {
+      this.param = param;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      TestClass testClass = (TestClass) o;
+      return Objects.equals(param, testClass.param);
+    }
+
+    @Override
+    public int hashCode() {
+
+      return Objects.hash(param);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoFactoryTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoFactoryTest.java
new file mode 100644
index 00000000000..a9e5e321198
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/KryoFactoryTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import com.esotericsoftware.kryo.Kryo;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** A set of unit {@link KryoFactory} tests. */
+public class KryoFactoryTest {
+
+  @Test
+  public void testGiveTheSameKrioAfterKryoRegistrarDeserialized()
+      throws IOException, ClassNotFoundException {
+
+    IdentifiedRegistrar registrar = IdentifiedRegistrar.of((k) -> k.register(TestClass.class));
+
+    Kryo firstKryo = KryoFactory.getOrCreateKryo(registrar);
+
+    ByteArrayOutputStream outStr = new ByteArrayOutputStream();
+    ObjectOutputStream oss = new ObjectOutputStream(outStr);
+
+    oss.writeObject(registrar);
+    oss.flush();
+    oss.close();
+
+    ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(outStr.toByteArray()));
+
+    @SuppressWarnings("unchecked")
+    IdentifiedRegistrar deserializedRegistrar = (IdentifiedRegistrar) ois.readObject();
+
+    Kryo secondKryo = KryoFactory.getOrCreateKryo(deserializedRegistrar);
+
+    Assert.assertSame(firstKryo, secondKryo);
+  }
+
+  private static class TestClass {}
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/RegisterCodersTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/RegisterCodersTest.java
new file mode 100644
index 00000000000..abfce78f195
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/coder/RegisterCodersTest.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.coder;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Unit test of {@link RegisterCoders}. */
+public class RegisterCodersTest {
+
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void testCodersRegistration() throws CannotProvideCoderException {
+
+    DummyTestCoder<FirstTestDataType> firstCoder = new DummyTestCoder<>();
+    DummyTestCoder<ParametrizedTestDataType<String>> parametrizedCoder = new DummyTestCoder<>();
+
+    RegisterCoders.to(pipeline)
+        .setKryoClassRegistrar(
+            (kryo) -> {
+              kryo.register(KryoSerializedTestType.class);
+            })
+        .registerCoder(FirstTestDataType.class, firstCoder)
+        .registerCoder(new TypeDescriptor<ParametrizedTestDataType<String>>() {}, parametrizedCoder)
+        .done();
+
+    CoderRegistry coderRegistry = pipeline.getCoderRegistry();
+
+    Assert.assertSame(firstCoder, coderRegistry.getCoder(FirstTestDataType.class));
+
+    Coder<KryoSerializedTestType> actualKryoTypeCoder =
+        coderRegistry.getCoder(KryoSerializedTestType.class);
+    Assert.assertTrue(actualKryoTypeCoder instanceof KryoCoder);
+
+    Coder<ParametrizedTestDataType<String>> parametrizedTypeActualCoder =
+        coderRegistry.getCoder(new TypeDescriptor<ParametrizedTestDataType<String>>() {});
+    Assert.assertSame(parametrizedCoder, parametrizedTypeActualCoder);
+  }
+
+  @Test
+  public void testKryoCoderTheSameSecondTime() throws CannotProvideCoderException {
+
+    RegisterCoders.to(pipeline)
+        .setKryoClassRegistrar(
+            (k) -> {
+              k.register(KryoSerializedTestType.class);
+            })
+        .done();
+
+    CoderRegistry coderRegistry = pipeline.getCoderRegistry();
+
+    Coder<KryoSerializedTestType> firstReturnedKryoTypeCoder =
+        coderRegistry.getCoder(KryoSerializedTestType.class);
+    Assert.assertTrue(firstReturnedKryoTypeCoder instanceof KryoCoder);
+
+    Coder<KryoSerializedTestType> secondReturnedKryoTypeCoder =
+        coderRegistry.getCoder(KryoSerializedTestType.class);
+    Assert.assertTrue(secondReturnedKryoTypeCoder instanceof KryoCoder);
+
+    Assert.assertSame(firstReturnedKryoTypeCoder, secondReturnedKryoTypeCoder);
+  }
+
+  @Test
+  public void testKryoRegisterCodeDecode() throws CannotProvideCoderException, IOException {
+
+    RegisterCoders.to(pipeline)
+        .setKryoClassRegistrar(
+            (k) -> {
+              k.register(SimpleDataClassWithEquality.class);
+            })
+        .done();
+
+    CoderRegistry coderRegistry = pipeline.getCoderRegistry();
+
+    Coder<SimpleDataClassWithEquality> coder =
+        coderRegistry.getCoder(SimpleDataClassWithEquality.class);
+    Assert.assertTrue(coder instanceof KryoCoder);
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+
+    SimpleDataClassWithEquality originalValue = new SimpleDataClassWithEquality("asdkjsadui");
+
+    coder.encode(originalValue, outputStream);
+
+    byte[] buf = outputStream.toByteArray();
+    ByteArrayInputStream inputStream = new ByteArrayInputStream(buf);
+
+    SimpleDataClassWithEquality decodedValue = coder.decode(inputStream);
+
+    Assert.assertNotNull(decodedValue);
+    Assert.assertEquals(originalValue, decodedValue);
+  }
+
+  private static class FirstTestDataType {}
+
+  private static class KryoSerializedTestType {}
+
+  private static class ParametrizedTestDataType<T> {}
+
+  private static class SimpleDataClassWithEquality {
+
+    private final String someField;
+
+    public SimpleDataClassWithEquality(String someField) {
+      this.someField = someField;
+    }
+
+    public String getSomeField() {
+      return someField;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      SimpleDataClassWithEquality that = (SimpleDataClassWithEquality) o;
+      return Objects.equals(someField, that.someField);
+    }
+
+    @Override
+    public int hashCode() {
+
+      return Objects.hash(someField);
+    }
+  }
+
+  private static class DummyTestCoder<T> extends Coder<T> {
+
+    @Override
+    public void encode(T value, OutputStream outStream) throws CoderException, IOException {
+      throwCoderException();
+    }
+
+    @Override
+    public T decode(InputStream inStream) throws CoderException, IOException {
+      throwCoderException();
+      return null;
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+      throwRuntimeException();
+      return null;
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      throwRuntimeException();
+    }
+
+    private void throwCoderException() throws CoderException {
+      throw new CoderException(
+          DummyTestCoder.class.getSimpleName() + " is supposed to do nothing.");
+    }
+
+    private void throwRuntimeException() throws RuntimeException {
+      throw new RuntimeException(
+          DummyTestCoder.class.getSimpleName() + " is supposed to do nothing.");
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java
new file mode 100644
index 00000000000..fce0968e45d
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/docs/DocumentationExamplesTest.java
@@ -0,0 +1,890 @@
+///*
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//package org.apache.beam.sdk.extensions.euphoria.core.docs;
+//
+//import static java.util.Arrays.asList;
+//
+//import com.google.common.base.Splitter;
+//import java.io.Serializable;
+//import java.util.Arrays;
+//import java.util.List;
+//import java.util.Optional;
+//import java.util.stream.Stream;
+//import org.apache.beam.sdk.Pipeline;
+//import org.apache.beam.sdk.coders.StringUtf8Coder;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CountByKey;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FullJoin;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.SumByKey;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.TopPerKey;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Util;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.util.Fold;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple;
+//import org.apache.beam.sdk.extensions.euphoria.core.translate.BeamFlow;
+//import org.apache.beam.sdk.extensions.euphoria.core.translate.EuphoriaPTransform;
+//import org.apache.beam.sdk.extensions.euphoria.core.translate.coder.KryoCoder;
+//import org.apache.beam.sdk.extensions.euphoria.core.translate.coder.RegisterCoders;
+//import org.apache.beam.sdk.extensions.euphoria.core.translate.common.PipelineUtils;
+//import org.apache.beam.sdk.io.TextIO;
+//import org.apache.beam.sdk.options.PipelineOptions;
+//import org.apache.beam.sdk.testing.PAssert;
+//import org.apache.beam.sdk.testing.TestPipeline;
+//import org.apache.beam.sdk.transforms.Create;
+//import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+//import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+//import org.apache.beam.sdk.values.KV;
+//import org.apache.beam.sdk.values.PCollection;
+//import org.apache.beam.sdk.values.TypeDescriptor;
+//import org.apache.beam.sdk.values.TypeDescriptors;
+//import org.joda.time.Duration;
+//import org.junit.Ignore;
+//import org.junit.Rule;
+//import org.junit.Test;
+//
+///**
+// * Contains all the examples from documentation page. Not all of them contains asserts, some do, but
+// * the rest is often here just to confirm that they compile. Once something break or changes, the
+// * documentation needs to change too.
+// */
+//public class DocumentationExamplesTest {
+//  private List<String> textLineByLine =
+//      Arrays.asList(
+//          "Lorem ipsum dolor sit amet, consectetur adipiscing elit. ",
+//          "Vestibulum volutpat pellentesque risus at sodales.",
+//          "Interdum et malesuada fames ac ante ipsum primis in faucibus.",
+//          "Donec sit amet arcu nec tellus sodales ultricies.",
+//          "Quisque ipsum fermentum nisl at libero accumsan consectetur.",
+//          "Praesent lobortis ex eget ex rhoncus, quis malesuada risus tristique.",
+//          "Aliquam risus at orci, porttitor eu turpis et, porttitor semper ligula.");
+//
+//  @Rule public final TestPipeline pipeline = TestPipeline.create();
+//
+//  @Ignore("We do not want to actually write output files from this test.")
+//  @Test
+//  public void wordCountExample() {
+//    PipelineOptions options = PipelineUtils.getDirectPipelineOptions();
+//
+//    Pipeline pipeline = Pipeline.create(options);
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    // Source of data loaded from Beam IO.
+//    PCollection<String> input =
+//        pipeline
+//            .apply(Create.of(textLineByLine))
+//            .setTypeDescriptor(TypeDescriptor.of(String.class));
+//    // Transform PCollection to euphoria's Dataset.
+//    Dataset<String> lines = flow.wrapped(input);
+//
+//    // FlatMap processes one input element at a time and allows user code to emit
+//    // zero, one, or more output elements. From input lines we will get data set of words.
+//    Dataset<String> words =
+//        FlatMap.named("TOKENIZER")
+//            .of(lines)
+//            .using(
+//                (String line, Collector<String> context) -> {
+//                  for (String word : Splitter.onPattern("\\s+").split(line)) {
+//                    context.collect(word);
+//                  }
+//                })
+//            .output();
+//
+//    // Now we can count input words - the operator ensures that all values for the same
+//    // key (word in this case) end up being processed together. Then it counts number of appearances
+//    // of the same key in 'words' dataset and emits it to output.
+//    Dataset<KV<String, Long>> counted = CountByKey.named("COUNT").of(words).keyBy(w -> w).output();
+//
+//    // Format output.
+//    Dataset<String> output =
+//        MapElements.named("FORMAT")
+//            .of(counted)
+//            .using(p -> p.getKey() + ": " + p.getValue())
+//            .output();
+//
+//    // Transform Dataset back to PCollection. It can be done in any step of this flow.
+//    PCollection<String> outputCollection = flow.unwrapped(output);
+//
+//    // Now we can again use Beam transformation. In this case we save words and their count
+//    // into the text file.
+//    outputCollection.apply(TextIO.write().to("counted_words"));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void inputsAndOutputsSection() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    PCollection<String> input =
+//        pipeline
+//            .apply(Create.of("mouse", "rat", "elephant", "cat", "X", "duck"))
+//            .setTypeDescriptor(TypeDescriptor.of(String.class));
+//
+//    Dataset<String> dataset = flow.wrapped(input);
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void addOperatorSection() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//    Dataset<Integer> input = flow.createInput(ListDataSource.bounded(asList(1, 2, 4, 3)));
+//
+//    Dataset<String> mappedElements =
+//        MapElements.named("Int2Str").of(input).using(String::valueOf).output();
+//
+//    PAssert.that(flow.unwrapped(mappedElements)).containsInAnyOrder("1", "2", "4", "3");
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void metricsAndAccumulatorsSection() {
+//    Pipeline pipeline = Pipeline.create(PipelineUtils.getDirectPipelineOptions());
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> dataset = Util.createMockDataset(flow, 1);
+//
+//    Dataset<String> flatMapped =
+//        FlatMap.named("FlatMap1")
+//            .of(dataset)
+//            .using(
+//                (String value, Collector<String> context) -> {
+//                  context.getCounter("my-counter").increment();
+//                  context.collect(value);
+//                })
+//            .output();
+//
+//    Dataset<String> mapped =
+//        MapElements.named("MapThem")
+//            .of(dataset)
+//            .using(
+//                (value, context) -> {
+//                  // use simple counter
+//                  context.getCounter("my-counter").increment();
+//
+//                  return value.toLowerCase();
+//                })
+//            .output();
+//  }
+//
+//  @Test
+//  public void codersAndTypesSection() {
+//    Pipeline pipeline = Pipeline.create(PipelineUtils.getDirectPipelineOptions());
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    KryoCoder<AnotherElementType> beamCoder = KryoCoder.withoutClassRegistration();
+//    KryoCoder<ParametrizedTestDataType<String>> typeParametrizedCoder =
+//        KryoCoder.withoutClassRegistration();
+//
+//    RegisterCoders.to(flow)
+//        .setKryoClassRegistrar(
+//            (kryo) -> {
+//              kryo.register(KryoSerializedElementType.class); //other may follow
+//            })
+//        .registerCoder(AnotherElementType.class, beamCoder)
+//        .registerCoder(
+//            new TypeDescriptor<ParametrizedTestDataType<String>>() {}, typeParametrizedCoder)
+//        .done();
+//
+//    Dataset<Integer> input = Util.createMockDataset(flow, 1);
+//
+//    MapElements.named("Int2Str")
+//        .of(input)
+//        .using(String::valueOf, TypeDescriptors.strings())
+//        .output();
+//  }
+//
+//  @Test
+//  public void windowingSection() {
+//
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    PCollection<Integer> inputPcoll =
+//        pipeline.apply(Create.of(1, 2, 3, 4)).setTypeDescriptor(TypeDescriptors.integers());
+//
+//    Dataset<Integer> input = flow.wrapped(inputPcoll);
+//
+//    Dataset<KV<Integer, Long>> countedElements =
+//        CountByKey.of(input)
+//            .keyBy(e -> e)
+//            .windowBy(FixedWindows.of(Duration.standardSeconds(1)))
+//            .triggeredBy(DefaultTrigger.of())
+//            .discardingFiredPanes()
+//            .output();
+//
+//    pipeline.run();
+//  }
+//
+//  private static class KryoSerializedElementType {}
+//
+//  private static class AnotherElementType {}
+//
+//  private static class ParametrizedTestDataType<T> {}
+//
+//  @Test
+//  public void countByKeyOperator() {
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> input = flow.createInput(ListDataSource.bounded(asList(1, 2, 4, 1, 1, 3)));
+//
+//    // suppose input: [1, 2, 4, 1, 1, 3]
+//    Dataset<KV<Integer, Long>> output = CountByKey.of(input).keyBy(e -> e).output();
+//    // Output will contain:  [KV(1, 3), KV(2, 1), KV(3, 1), (4, 1)]
+//
+//    PAssert.that(flow.unwrapped(output))
+//        .containsInAnyOrder(asList(KV.of(1, 3L), KV.of(2, 1L), KV.of(3, 1L), KV.of(4, 1L)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void distinctOperator() {
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> input = flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 3, 2, 1)));
+//
+//    // suppose input: [1, 2, 3, 3, 2, 1]
+//    Distinct.named("unique-integers-only").of(input).output();
+//    // Output will contain:  1, 2, 3
+//
+//    Dataset<KV<Integer, Long>> keyValueInput =
+//        flow.createInput(
+//            ListDataSource.bounded(
+//                asList(
+//                    KV.of(1, 100L),
+//                    KV.of(3, 100_000L),
+//                    KV.of(42, 10L),
+//                    KV.of(1, 0L),
+//                    KV.of(3, 0L))));
+//
+//    // suppose input: [KV(1, 100L), KV(3, 100_000L), KV(42, 10L), KV(1, 0L), KV(3, 0L)]
+//    Dataset<Integer> uniqueKeys =
+//        Distinct.named("unique-keys-only").of(keyValueInput).mapped(KV::getKey).output();
+//    // Output will contain:  1, 3, 42
+//
+//    PAssert.that(flow.unwrapped(uniqueKeys)).containsInAnyOrder(1, 3, 42);
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void batchJoinOperator() {
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> left = flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 0, 4, 3, 1)));
+//    Dataset<String> right =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    // suppose that left contains: [1, 2, 3, 0, 4, 3, 1]
+//    // suppose that right contains: ["mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, String>> joined =
+//        Join.named("join-length-to-words")
+//            .of(left, right)
+//            .by(le -> le, String::length) // key extractors
+//            .using((Integer l, String r, Collector<String> c) -> c.collect(l + "+" + r))
+//            .output();
+//
+//    // joined will contain: [ KV(1, "1+X"), KV(3, "3+cat"), KV(3, "3+rat"), KV(4, "4+duck"),
+//    // KV(3, "3+cat"), KV(3, "3+rat"), KV(1, "1+X")]
+//
+//    PCollection<KV<Integer, String>> outputPCollection = flow.unwrapped(joined);
+//    PAssert.that(outputPCollection)
+//        .containsInAnyOrder(
+//            asList(
+//                KV.of(1, "1+X"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(4, "4+duck"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(1, "1+X")));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void batchLeftJoinOperator() {
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> left = flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 0, 4, 3, 1)));
+//    Dataset<String> right =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    // suppose that left contains: [1, 2, 3, 0, 4, 3, 1]
+//    // suppose that right contains: ["mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, String>> joined =
+//        LeftJoin.named("left-join-length-to-words")
+//            .of(left, right)
+//            .by(le -> le, String::length) // key extractors
+//            .using(
+//                (Integer l, Optional<String> r, Collector<String> c) ->
+//                    c.collect(l + "+" + r.orElse(null)))
+//            .output();
+//
+//    // joined will contain: [KV(1, "1+X"), KV(2, "2+null"), KV(3, "3+cat"),
+//    // KV(3, "3+rat"), KV(0, "0+null"), KV(4, "4+duck"), KV(3, "3+cat"),
+//    // KV(3, "3+rat"), KV(1, "1+X")]
+//
+//    PCollection<KV<Integer, String>> outputPCollection = flow.unwrapped(joined);
+//    PAssert.that(outputPCollection)
+//        .containsInAnyOrder(
+//            asList(
+//                KV.of(1, "1+X"),
+//                KV.of(2, "2+null"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(0, "0+null"),
+//                KV.of(4, "4+duck"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(1, "1+X")));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void batchRightJoinFullOperator() {
+//
+//    // Transform to euphoria's flow.
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> left = flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 0, 4, 3, 1)));
+//    Dataset<String> right =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    // suppose that left contains: [1, 2, 3, 0, 4, 3, 1]
+//    // suppose that right contains: ["mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, String>> joined =
+//        RightJoin.named("right-join-length-to-words")
+//            .of(left, right)
+//            .by(le -> le, String::length) // key extractors
+//            .using(
+//                (Optional<Integer> l, String r, Collector<String> c) ->
+//                    c.collect(l.orElse(null) + "+" + r))
+//            .output();
+//
+//    // joined will contain: [ KV(1, "1+X"), KV(3, "3+cat"), KV(3, "3+rat"),
+//    // KV(4, "4+duck"), KV(3, "3+cat"), KV(3, "3+rat"), KV(1, "1+X"),
+//    // KV(8, "null+elephant"), KV(5, "null+mouse")]
+//
+//    PCollection<KV<Integer, String>> outputPCollection = flow.unwrapped(joined);
+//    PAssert.that(outputPCollection)
+//        .containsInAnyOrder(
+//            asList(
+//                KV.of(1, "1+X"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(4, "4+duck"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(1, "1+X"),
+//                KV.of(8, "null+elephant"),
+//                KV.of(5, "null+mouse")));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void batchFullJoinOperator() {
+//
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> left = flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 0, 4, 3, 1)));
+//    Dataset<String> right =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    // suppose that left contains: [1, 2, 3, 0, 4, 3, 1]
+//    // suppose that right contains: ["mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, String>> joined =
+//        FullJoin.named("join-length-to-words")
+//            .of(left, right)
+//            .by(le -> le, String::length) // key extractors
+//            .using(
+//                (Optional<Integer> l, Optional<String> r, Collector<String> c) ->
+//                    c.collect(l.orElse(null) + "+" + r.orElse(null)))
+//            .output();
+//
+//    // joined will contain: [ KV(1, "1+X"), KV(2, "2+null"), KV(3, "3+cat"), KV(3, "3+rat"),
+//    // KV(0, "0+null"), KV(4, "4+duck"), KV(3, "3+cat"), KV(3, "3+rat"),KV(1, "1+X"),
+//    //  KV(1, "null+elephant"), KV(5, "null+mouse")];
+//
+//    PCollection<KV<Integer, String>> outputPCollection = flow.unwrapped(joined);
+//    PAssert.that(outputPCollection)
+//        .containsInAnyOrder(
+//            asList(
+//                KV.of(1, "1+X"),
+//                KV.of(2, "2+null"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(0, "0+null"),
+//                KV.of(4, "4+duck"),
+//                KV.of(3, "3+cat"),
+//                KV.of(3, "3+rat"),
+//                KV.of(1, "1+X"),
+//                KV.of(8, "null+elephant"),
+//                KV.of(5, "null+mouse")));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void mapElementsOperator() {
+//
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//    Dataset<Integer> input = flow.createInput(ListDataSource.bounded(asList(0, 1, 2, 3, 4, 5)));
+//
+//    // suppose inputs contains: [ 0, 1, 2, 3, 4, 5]
+//    Dataset<String> strings = MapElements.named("int2str").of(input).using(i -> "#" + i).output();
+//    // strings will contain: [ "#0", "#1", "#2", "#3", "#4", "#5"]
+//
+//    PAssert.that(flow.unwrapped(strings)).containsInAnyOrder("#0", "#1", "#2", "#3", "#4", "#5");
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void flatMapOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> words =
+//        flow.createInput(ListDataSource.bounded(asList("Brown", "fox", ".", "")));
+//
+//    // suppose words contain: ["Brown", "fox", ".", ""]
+//    Dataset<String> letters =
+//        FlatMap.named("str2char")
+//            .of(words)
+//            .using(
+//                (String s, Collector<String> collector) -> {
+//                  for (int i = 0; i < s.length(); i++) {
+//                    char c = s.charAt(i);
+//                    collector.collect(String.valueOf(c));
+//                  }
+//                })
+//            .output();
+//    // characters will contain: ["B", "r", "o", "w", "n",  "f", "o", "x", "."]
+//
+//    PAssert.that(flow.unwrapped(letters))
+//        .containsInAnyOrder("B", "r", "o", "w", "n", "f", "o", "x", ".");
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void flatMapWithTimeExtractorOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<SomeEventObject> events =
+//        flow.createInput(
+//            ListDataSource.bounded(
+//                asList(
+//                    new SomeEventObject(0),
+//                    new SomeEventObject(1),
+//                    new SomeEventObject(2),
+//                    new SomeEventObject(3),
+//                    new SomeEventObject(4))));
+//
+//    // suppose events contain events of SomeEventObject, its 'getEventTimeInMillis()' methods returns time-stamp
+//    Dataset<SomeEventObject> timeStampedEvents =
+//        FlatMap.named("extract-event-time")
+//            .of(events)
+//            .using((SomeEventObject e, Collector<SomeEventObject> c) -> c.collect(e))
+//            .eventTimeBy(SomeEventObject::getEventTimeInMillis)
+//            .output();
+//    //Euphoria will now know event time for each event
+//
+//    //        PAssert.that(flow.unwrapped(timeStampedEvents))
+//    //            .inWindow(new IntervalWindow(new Instant(0), new Instant(5)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void filterOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> nums =
+//        flow.createInput(ListDataSource.bounded(asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)));
+//
+//    // suppose nums contains: [0,  1, 2, 3, 4, 5, 6, 7, 8, 9]
+//    Dataset<Integer> divisibleBythree =
+//        Filter.named("divisibleByFive").of(nums).by(e -> e % 3 == 0).output();
+//    //divisibleBythree will contain: [ 0, 3, 6, 9]
+//
+//    PAssert.that(flow.unwrapped(divisibleBythree)).containsInAnyOrder(0, 3, 6, 9);
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void reduceByKeyTestOperator1() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> animals =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    //suppose animals contains : [ "mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, Long>> countOfAnimalNamesByLength =
+//        ReduceByKey.named("to-letters-couts")
+//            .of(animals)
+//            .keyBy(String::length) // length of animal name will be used as groupping key
+//            // we need to count each animal name once, so why not to optimize each string to 1
+//            .valueBy(e -> 1)
+//            .reduceBy(Stream::count)
+//            .output();
+//    // countOfAnimalNamesByLength wil contain [ KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L) ]
+//
+//    PAssert.that(flow.unwrapped(countOfAnimalNamesByLength))
+//        .containsInAnyOrder(
+//            asList(KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void reduceByKeyTestOperatorCombinable() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> animals =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    //suppose animals contains : [ "mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, Long>> countOfAnimalNamesByLenght =
+//        ReduceByKey.named("to-letters-couts")
+//            .of(animals)
+//            .keyBy(String::length) // length of animal name will be used as grouping key
+//            // we need to count each animal name once, so why not to optimize each string to 1
+//            .valueBy(e -> 1L)
+//            .combineBy(s -> s.mapToLong(l -> l).sum())
+//            .output();
+//    // countOfAnimalNamesByLength wil contain [ KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L) ]
+//
+//    PAssert.that(flow.unwrapped(countOfAnimalNamesByLenght))
+//        .containsInAnyOrder(
+//            asList(KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void reduceByKeyTestOperatorContext() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> animals =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    //suppose animals contains : [ "mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, Long>> countOfAnimalNamesByLenght =
+//        ReduceByKey.named("to-letters-couts")
+//            .of(animals)
+//            .keyBy(String::length) // length of animal name will e used as grouping key
+//            // we need to count each animal name once, so why not to optimize each string to 1
+//            .valueBy(e -> 1)
+//            .reduceBy(
+//                (Stream<Integer> s, Collector<Long> collector) -> {
+//                  collector.collect(s.count());
+//                  collector.asContext().getCounter("num-of-keys").increment();
+//                })
+//            .output();
+//    // countOfAnimalNamesByLength wil contain [ KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L) ]
+//
+//    PAssert.that(flow.unwrapped(countOfAnimalNamesByLenght))
+//        .containsInAnyOrder(
+//            asList(KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L)));
+//
+//    pipeline.run();
+//  }
+//
+//  /**
+//   * Note that this one is not mentioned in documentation due to high number of RBK examples and
+//   * rather lower explanation value. Please consider to include it in future
+//   */
+//  @Test
+//  public void reduceByKeyTestOperatorContextManyOutputs() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> animals =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    Dataset<KV<Integer, Long>> countOfAnimalNamesByLenght =
+//        ReduceByKey.named("to-letters-couts")
+//            .of(animals)
+//            .keyBy(String::length) // length of animal name will e used as grouping key
+//            // we need to count each animal name once, so why not to optimize each string to 1
+//            .valueBy(e -> 1)
+//            .reduceBy(
+//                (Stream<Integer> s, Collector<Long> collector) -> {
+//                  long count = s.count();
+//                  collector.collect(count);
+//                  collector.collect(2L * count);
+//                })
+//            .output();
+//
+//    PAssert.that(flow.unwrapped(countOfAnimalNamesByLenght))
+//        .containsInAnyOrder(
+//            asList(
+//                KV.of(1, 1L),
+//                KV.of(3, 2L),
+//                KV.of(4, 1L),
+//                KV.of(5, 1L),
+//                KV.of(8, 1L),
+//                KV.of(1, 2L),
+//                KV.of(3, 4L),
+//                KV.of(4, 2L),
+//                KV.of(5, 2L),
+//                KV.of(8, 2L)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void reduceByKeyTestOperatorFold() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> animals =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("mouse", "rat", "elephant", "cat", "X", "duck")));
+//
+//    //suppose animals contains : [ "mouse", "rat", "elephant", "cat", "X", "duck"]
+//    Dataset<KV<Integer, Long>> countOfAnimalNamesByLenght =
+//        ReduceByKey.named("to-letters-couts")
+//            .of(animals)
+//            .keyBy(String::length) // length of animal name will be used as grouping key
+//            // we need to count each animal name once, so why not to optimize each string to 1
+//            .valueBy(e -> 1L)
+//            .combineBy(Fold.of((l1, l2) -> l1 + l2))
+//            .output();
+//    // countOfAnimalNamesByLength will contain [ KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L) ]
+//
+//    PAssert.that(flow.unwrapped(countOfAnimalNamesByLenght))
+//        .containsInAnyOrder(
+//            asList(KV.of(1, 1L), KV.of(3, 2L), KV.of(4, 1L), KV.of(5, 1L), KV.of(8, 1L)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void testSumByKeyOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//    Dataset<Integer> input =
+//        flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 4, 5, 6, 7, 8, 9)));
+//
+//    //suppose input contains: [ 1, 2, 3, 4, 5, 6, 7, 8, 9 ]
+//    Dataset<KV<Integer, Long>> output =
+//        SumByKey.named("sum-odd-and-even")
+//            .of(input)
+//            .keyBy(e -> e % 2)
+//            .valueBy(e -> (long) e)
+//            .output();
+//    // output will contain: [ KV.of(0, 20L), KV.of(1, 25L)]
+//
+//    PAssert.that(flow.unwrapped(output)).containsInAnyOrder(asList(KV.of(0, 20L), KV.of(1, 25L)));
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void testUnionOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> cats =
+//        flow.createInput(ListDataSource.bounded(asList("cheetah", "cat", "lynx", "jaguar")));
+//
+//    Dataset<String> rodents =
+//        flow.createInput(
+//            ListDataSource.bounded(asList("squirrel", "mouse", "rat", "lemming", "beaver")));
+//
+//    //suppose cats contains: [ "cheetah", "cat", "lynx", "jaguar" ]
+//    //suppose rodents conains: [ "squirrel", "mouse", "rat", "lemming", "beaver" ]
+//    Dataset<String> animals = Union.named("to-animals").of(cats, rodents).output();
+//
+//    // animal will contain: "cheetah", "cat", "lynx", "jaguar", "squirrel", "mouse", "rat", "lemming", "beaver"
+//    PAssert.that(flow.unwrapped(animals))
+//        .containsInAnyOrder(
+//            "cheetah", "cat", "lynx", "jaguar", "squirrel", "mouse", "rat", "lemming", "beaver");
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void testAssignEventTimeOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<SomeEventObject> events =
+//        flow.createInput(
+//            ListDataSource.bounded(
+//                asList(
+//                    new SomeEventObject(0),
+//                    new SomeEventObject(1),
+//                    new SomeEventObject(2),
+//                    new SomeEventObject(3),
+//                    new SomeEventObject(4))));
+//
+//    // suppose events contain events of SomeEventObject, its 'getEventTimeInMillis()' methods returns time-stamp
+//    Dataset<SomeEventObject> timeStampedEvents =
+//        AssignEventTime.named("extract-event-time")
+//            .of(events)
+//            .using(SomeEventObject::getEventTimeInMillis)
+//            .output();
+//    //Euphoria will now know event time for each event
+//
+//    //    PAssert.that(flow.unwrapped(timeStampedEvents))
+//    //        .inWindow(new IntervalWindow(new Instant(0), new Instant(5)));
+//
+//    pipeline.run();
+//  }
+//
+//  private static class SomeEventObject implements Serializable {
+//
+//    private long timestamp;
+//
+//    SomeEventObject(long timestamp) {
+//      this.timestamp = timestamp;
+//    }
+//
+//    long getEventTimeInMillis() {
+//      return timestamp;
+//    }
+//  }
+//
+//  @Test
+//  public void testEuphoriaPTransformExample() {
+//
+//    PCollection<String> inputs =
+//        pipeline.apply(
+//            "Create", Create.of("a", "b", "c", "A", "a", "C", "x").withCoder(StringUtf8Coder.of()));
+//
+//    //suppose inputs PCollection contains: [ "a", "b", "c", "A", "a", "C", "x"]
+//    PCollection<KV<String, Long>> lettersWithCounts =
+//        inputs.apply(
+//            "count-uppercase-letters-in-Euphoria",
+//            EuphoriaPTransform.of(
+//                (Dataset<String> input) -> {
+//                  Dataset<String> upperCase =
+//                      MapElements.of(input)
+//                          .using((UnaryFunction<String, String>) String::toUpperCase)
+//                          .output();
+//
+//                  return CountByKey.of(upperCase).keyBy(e -> e).output();
+//                }));
+//    //now the 'lettersWithCounts' will contain [ KV("A", 3L), KV("B", 1L), KV("C", 2L), KV("X", 1L) ]
+//
+//    PAssert.that(lettersWithCounts)
+//        .containsInAnyOrder(asList(KV.of("A", 3L), KV.of("B", 1L), KV.of("C", 2L), KV.of("X", 1L)));
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void testReduceWithWindowOperator() {
+//
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<Integer> input =
+//        flow.createInput(ListDataSource.bounded(asList(1, 2, 3, 4, 5, 6, 7, 8)));
+//
+//    //suppose input contains [ 1, 2, 3, 4, 5, 6, 7, 8 ]
+//    //lets assign time-stamp to each input element
+//    Dataset<Integer> withEventTime = AssignEventTime.of(input).using(i -> 1000L * i).output();
+//
+//    Dataset<Integer> output =
+//        ReduceWindow.of(withEventTime)
+//            .combineBy(Fold.of((i1, i2) -> i1 + i2))
+//            .windowBy(FixedWindows.of(Duration.millis(5000)))
+//            .triggeredBy(DefaultTrigger.of())
+//            .discardingFiredPanes()
+//            .output();
+//    //output will contain: [ 10, 26 ]
+//
+//    PAssert.that(flow.unwrapped(output)).containsInAnyOrder(10, 26);
+//
+//    pipeline.run();
+//  }
+//
+//  @Test
+//  public void testTopPerKeyOperator() {
+//    BeamFlow flow = BeamFlow.of(pipeline);
+//
+//    Dataset<String> animals =
+//        flow.createInput(
+//            ListDataSource.bounded(
+//                asList(
+//                    "mouse",
+//                    "elk",
+//                    "rat",
+//                    "mule",
+//                    "elephant",
+//                    "dinosaur",
+//                    "cat",
+//                    "duck",
+//                    "caterpillar")));
+//
+//    // suppose 'animals contain: [ "mouse", "elk", "rat", "mule", "elephant", "dinosaur", "cat", "duck", "caterpillar" ]
+//    Dataset<Triple<Character, String, Integer>> longestNamesByLetter =
+//        TopPerKey.named("longest-animal-names")
+//            .of(animals)
+//            .keyBy(name -> name.charAt(0)) // first character is the key
+//            .valueBy(UnaryFunction.identity()) // value type is the same as input element type
+//            .scoreBy(
+//                String
+//                    ::length) // length defines score, note that Integer implements Comparable<Integer>
+//            .output();
+//    //longestNamesByLetter wil contain: [ ('m', "mouse", 5), ('r', "rat", 3), ('e', "elephant", 8), ('d', "dinosaur", 8), ('c', "caterpillar", 11) ]
+//
+//    PAssert.that(flow.unwrapped(longestNamesByLetter))
+//        .containsInAnyOrder(
+//            Triple.of('m', "mouse", 5),
+//            Triple.of('r', "rat", 3),
+//            Triple.of('e', "elephant", 8),
+//            Triple.of('d', "dinosaur", 8),
+//            Triple.of('c', "caterpillar", 11));
+//
+//    pipeline.run();
+//  }
+//}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/AbstractOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/AbstractOperatorTest.java
new file mode 100644
index 00000000000..8a6c7053370
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/AbstractOperatorTest.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.coder.KryoCoder;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SingleJvmAccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SnapshotProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.EuphoriaOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** Base class for test description of a test case. */
+public abstract class AbstractOperatorTest implements Serializable {
+
+  /**
+   * Run all tests with given runner.
+   *
+   * @param tc the test case to executeSync
+   */
+  @SuppressWarnings("unchecked")
+  public <T> void execute(TestCase<T> tc) {
+
+    final SingleJvmAccumulatorProvider.Factory accumulatorProvider =
+        SingleJvmAccumulatorProvider.Factory.get();
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    final EuphoriaOptions euphoriaOptions = pipelineOptions.as(EuphoriaOptions.class);
+    euphoriaOptions.setAccumulatorProviderFactory(accumulatorProvider);
+    final Pipeline pipeline = TestPipeline.create(pipelineOptions);
+    pipeline
+        .getCoderRegistry()
+        .registerCoderForClass(Object.class, KryoCoder.withoutClassRegistration());
+    final Dataset<T> output = tc.getOutput(pipeline);
+    tc.validate(output);
+    pipeline.run().waitUntilFinish();
+    tc.validateAccumulators(accumulatorProvider);
+  }
+
+  /** A single test case. */
+  protected interface TestCase<T> extends Serializable {
+
+    /**
+     * Retrieve flow to be run. Write outputs to given sink.
+     *
+     * @param pipeline the flow to attach the test logic to
+     * @return the output data set representing the result of the test logic
+     */
+    Dataset<T> getOutput(Pipeline pipeline);
+
+    /**
+     * Retrieve expected outputs.
+     *
+     * <p>These outputs will be compared irrespective of order.
+     *
+     * @return list of expected outputs that will be compared irrespective of order
+     */
+    default List<T> getUnorderedOutput() {
+      throw new UnsupportedOperationException(
+          "Override either `getUnorderedOutput()`, or `validate`");
+    }
+
+    /**
+     * Validate that the raw output is correct.
+     *
+     * @param outputs the raw outputs produced by sink
+     * @throws AssertionError when the output is not correct
+     */
+    default void validate(Dataset<T> outputs) throws AssertionError {
+      PAssert.that(outputs.getPCollection()).containsInAnyOrder(getUnorderedOutput());
+    }
+
+    /**
+     * Validate accumulators given a provider capturing the accumulated values.
+     *
+     * @param snapshots the provider of the accumulated values
+     */
+    default void validateAccumulators(SnapshotProvider snapshots) {}
+  }
+
+  /** Abstract {@code TestCase} to be extended by test classes. */
+  public abstract static class AbstractTestCase<InputT, OutputT> implements TestCase<OutputT> {
+
+    @Override
+    public final Dataset<OutputT> getOutput(Pipeline pipeline) {
+      final List<InputT> inputData = getInput();
+      final Dataset<InputT> inputDataset =
+          Dataset.of(
+              pipeline.apply("input", Create.of(inputData)).setTypeDescriptor(getInputType()));
+      return getOutput(inputDataset);
+    }
+
+    protected abstract TypeDescriptor<InputT> getInputType();
+
+    protected abstract Dataset<OutputT> getOutput(Dataset<InputT> input);
+
+    protected abstract List<InputT> getInput();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/BroadcastHashJoinTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/BroadcastHashJoinTest.java
new file mode 100644
index 00000000000..aee805f035e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/BroadcastHashJoinTest.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.BroadcastHashJoinTranslator;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.EuphoriaOptions;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.FlatMapTranslator;
+import org.apache.beam.sdk.extensions.euphoria.core.translate.SimpleTranslatorProvider;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Collection of broadcast hash join tests. */
+public class BroadcastHashJoinTest extends AbstractOperatorTest {
+
+  private abstract static class TestCase<LeftT, RightT, OutputT>
+      extends JoinTest.JoinTestCase<LeftT, RightT, OutputT> {
+
+    @Override
+    public Dataset<OutputT> getOutput(Pipeline pipeline) {
+      pipeline
+          .getOptions()
+          .as(EuphoriaOptions.class)
+          .setTranslatorProvider(
+              SimpleTranslatorProvider.newBuilder()
+                  .registerTranslator(FlatMap.class, new FlatMapTranslator<>())
+                  .registerTranslator(Join.class, new BroadcastHashJoinTranslator<>())
+                  .build());
+      return super.getOutput(pipeline);
+    }
+  }
+
+  @Test
+  public void leftBroadcastHashJoin() {
+    execute(
+        new TestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return LeftJoin.of(left, MapElements.of(right).using(i -> i).output())
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Integer l, Optional<Long> r, Collector<String> c) ->
+                        c.collect(l + "+" + r.orElse(null)))
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L, 11L);
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, "0+null"),
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"));
+          }
+        });
+  }
+
+  @Test
+  public void rightBroadcastHashJoin() {
+    execute(
+        new TestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return RightJoin.of(MapElements.of(left).using(i -> i).output(), right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Long r, Collector<String> c) ->
+                        c.collect(l.orElse(null) + "+" + r))
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"),
+                KV.of(5, "null+15"));
+          }
+        });
+  }
+
+  @Test
+  public void keyHashCollisionBroadcastHashJoin() {
+    final String sameHashCodeKey1 = "FB";
+    final String sameHashCodeKey2 = "Ea";
+    execute(
+        new TestCase<String, Integer, KV<String, String>>() {
+
+          @Override
+          protected Dataset<KV<String, String>> getOutput(
+              Dataset<String> left, Dataset<Integer> right) {
+            return LeftJoin.of(left, MapElements.of(right).using(i -> i).output())
+                .by(e -> e, e -> e % 2 == 0 ? sameHashCodeKey2 : sameHashCodeKey1)
+                .using(
+                    (String l, Optional<Integer> r, Collector<String> c) ->
+                        c.collect(l + "+" + r.orElse(null)))
+                .output();
+          }
+
+          @Override
+          protected List<String> getLeftInput() {
+            return Arrays.asList(sameHashCodeKey1, sameHashCodeKey2, "keyWithoutRightSide");
+          }
+
+          @Override
+          protected TypeDescriptor<String> getLeftInputType() {
+            return TypeDescriptors.strings();
+          }
+
+          @Override
+          protected List<Integer> getRightInput() {
+            return Arrays.asList(1, 2);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getRightInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<KV<String, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(sameHashCodeKey1, "FB+1"),
+                KV.of(sameHashCodeKey2, "Ea+2"),
+                KV.of("keyWithoutRightSide", "keyWithoutRightSide+null"));
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/CountByKeyTest.java
new file mode 100644
index 00000000000..2d73ce4effa
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/CountByKeyTest.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CountByKey;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Test operator {@code CountByKey}. */
+public class CountByKeyTest extends AbstractOperatorTest {
+
+  @Test
+  public void testCount() {
+    execute(
+        new AbstractTestCase<Integer, KV<Integer, Long>>() {
+          @Override
+          protected Dataset<KV<Integer, Long>> getOutput(Dataset<Integer> input) {
+            // ~ use stable event-time watermark
+            input = AssignEventTime.of(input).using(e -> 0).output();
+            return CountByKey.of(input)
+                .keyBy(e -> e)
+                .windowBy(FixedWindows.of(Duration.standardSeconds(1)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 10, 9, 8, 7, 6, 5, 4);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<KV<Integer, Long>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(2, 1L),
+                KV.of(4, 2L),
+                KV.of(6, 2L),
+                KV.of(8, 1L),
+                KV.of(10, 1L),
+                KV.of(1, 1L),
+                KV.of(3, 1L),
+                KV.of(5, 2L),
+                KV.of(7, 2L),
+                KV.of(9, 1L));
+          }
+        });
+  }
+
+  @Test
+  public void testWithEventTimeWindow() {
+    execute(
+        new AbstractTestCase<KV<Integer, Long>, KV<Integer, Long>>() {
+
+          @Override
+          protected Dataset<KV<Integer, Long>> getOutput(Dataset<KV<Integer, Long>> input) {
+            input = AssignEventTime.of(input).using(KV::getValue).output();
+            return CountByKey.of(input)
+                .keyBy(KV::getKey)
+                .windowBy(FixedWindows.of(Duration.standardSeconds(1)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          protected List<KV<Integer, Long>> getInput() {
+            return Arrays.asList(
+                KV.of(1, 200L),
+                KV.of(2, 500L),
+                KV.of(1, 800L),
+                KV.of(3, 1400L),
+                KV.of(3, 1200L),
+                KV.of(4, 1800L),
+                KV.of(5, 2100L),
+                KV.of(5, 2300L),
+                KV.of(5, 2700L),
+                KV.of(5, 3500L),
+                KV.of(5, 3300L),
+                KV.of(6, 3800L),
+                KV.of(7, 4400L),
+                KV.of(7, 4500L),
+                KV.of(10, 4600L),
+                KV.of(10, 5100L),
+                KV.of(9, 5200L),
+                KV.of(9, 5500L),
+                KV.of(9, 6300L),
+                KV.of(9, 6700L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<Integer, Long>> getInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.longs());
+          }
+
+          @Override
+          public List<KV<Integer, Long>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(1, 2L),
+                KV.of(2, 1L),
+                KV.of(3, 2L),
+                KV.of(4, 1L),
+                KV.of(5, 3L),
+                KV.of(5, 2L),
+                KV.of(6, 1L),
+                KV.of(7, 2L),
+                KV.of(10, 1L),
+                KV.of(10, 1L),
+                KV.of(9, 2L),
+                KV.of(9, 2L));
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/DistinctTest.java
new file mode 100644
index 00000000000..87e014a09a5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/DistinctTest.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test for the {@link Distinct} operator. */
+public class DistinctTest extends AbstractOperatorTest {
+
+  /** Test simple duplicates. */
+  @Test
+  public void testSimpleDuplicatesWithNoWindowing() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 2, 3);
+          }
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            return Distinct.of(input).output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+        });
+  }
+
+  /** Test simple duplicates with unbounded input with count window. */
+  @Test
+  public void testSimpleDuplicatesWithTimeWindowing() {
+    execute(
+        new AbstractTestCase<KV<Integer, Long>, Integer>() {
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 2, 3, 2, 1);
+          }
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<KV<Integer, Long>> input) {
+            input = AssignEventTime.of(input).using(KV::getValue).output();
+            return Distinct.of(input)
+                .mapped(KV::getKey)
+                .windowBy(FixedWindows.of(org.joda.time.Duration.standardSeconds(1)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          protected List<KV<Integer, Long>> getInput() {
+            return Arrays.asList(
+                KV.of(1, 100L),
+                KV.of(2, 300L), // first window
+                KV.of(3, 1200L),
+                KV.of(3, 1500L), // second window
+                KV.of(2, 2200L),
+                KV.of(1, 2700L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<Integer, Long>> getInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.longs());
+          }
+        });
+  }
+
+  @Test
+  public void testSimpleDuplicatesWithStream() {
+    execute(
+        new AbstractTestCase<KV<Integer, Long>, Integer>() {
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(2, 1, 3);
+          }
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<KV<Integer, Long>> input) {
+            input = AssignEventTime.of(input).using(KV::getValue).output();
+            return Distinct.of(input)
+                .mapped(KV::getKey)
+                .windowBy(FixedWindows.of(org.joda.time.Duration.standardSeconds(1)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          protected List<KV<Integer, Long>> getInput() {
+            List<KV<Integer, Long>> first = asTimedList(100, 1, 2, 3, 3, 2, 1);
+            first.addAll(asTimedList(100, 1, 2, 3, 3, 2, 1));
+            return first;
+          }
+
+          @Override
+          protected TypeDescriptor<KV<Integer, Long>> getInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.longs());
+          }
+        });
+  }
+
+  private List<KV<Integer, Long>> asTimedList(long step, Integer... values) {
+    final List<KV<Integer, Long>> ret = new ArrayList<>(values.length);
+    long i = step;
+    for (Integer v : values) {
+      ret.add(KV.of(v, i));
+      i += step;
+    }
+    return ret;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FilterTest.java
new file mode 100644
index 00000000000..59cb2e8a944
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FilterTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test operator {@code Filter}. */
+public class FilterTest extends AbstractOperatorTest {
+
+  @Test
+  public void testFilter() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            return Filter.of(input).by(e -> e % 2 == 0).output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(2, 4, 6, 8, 10, 12, 14);
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java
new file mode 100644
index 00000000000..d70bc29951b
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SnapshotProvider;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test operator {@code FlatMap}. */
+public class FlatMapTest extends AbstractOperatorTest {
+
+  @Test
+  public void testExplodeOnTwoPartitions() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            return FlatMap.of(input)
+                .using(
+                    (Integer e, Collector<Integer> c) -> {
+                      for (int i = 1; i <= e; i++) {
+                        c.collect(i);
+                      }
+                    })
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 1, 2, 1, 2, 3, 1, 2, 3, 4, 1, 2, 3, 1, 2, 1);
+          }
+        });
+  }
+
+  @Test
+  public void testCounterTest() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 0, 10, 20);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            return FlatMap.named("test")
+                .of(input)
+                .using(
+                    (UnaryFunctor<Integer, Integer>)
+                        (elem, collector) -> {
+                          collector.getCounter("input").increment();
+                          collector.getCounter("sum").increment(elem);
+                          collector.collect(elem * elem);
+                        })
+                .output();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 4, 9, 16, 25, 36, 0, 100, 400);
+          }
+
+          @Override
+          public void validateAccumulators(SnapshotProvider snapshots) {
+            Map<String, Long> counters = snapshots.getCounterSnapshots();
+            assertEquals(Long.valueOf(9L), counters.get("input"));
+            assertEquals(Long.valueOf(51L), counters.get("sum"));
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java
new file mode 100644
index 00000000000..7a095aca1c8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java
@@ -0,0 +1,843 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FullJoin;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin;
+import org.apache.beam.sdk.extensions.euphoria.core.coder.KryoCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+/** Test operator {@code Join}. */
+public class JoinTest extends AbstractOperatorTest {
+
+  @Test
+  public void batchJoinFullOuter() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return FullJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Optional<Long> r, Collector<String> c) ->
+                        c.collect(l.orElse(null) + "+" + r.orElse(null)))
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, "0+null"),
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"),
+                KV.of(5, "null+15"));
+          }
+        });
+  }
+
+  @Test
+  public void batchJoinFullOuterExample() {
+    execute(
+        new JoinTestCase<Integer, String, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<String> right) {
+
+            return FullJoin.of(left, right)
+                .by(le -> le, String::length)
+                .using(
+                    (Optional<Integer> l, Optional<String> r, Collector<String> c) ->
+                        c.collect(l.orElse(null) + "+" + r.orElse(null)))
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<String> getRightInput() {
+            return Arrays.asList("mouse", "rat", "cat", "X", "duck");
+          }
+
+          @Override
+          protected TypeDescriptor<String> getRightInputType() {
+            return TypeDescriptors.strings();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(1, "1+X"),
+                KV.of(2, "2+null"),
+                KV.of(3, "3+cat"),
+                KV.of(3, "3+rat"),
+                KV.of(0, "0+null"),
+                KV.of(4, "4+duck"),
+                KV.of(3, "3+cat"),
+                KV.of(3, "3+rat"),
+                KV.of(1, "1+X"),
+                KV.of(5, "null+mouse"));
+          }
+        });
+  }
+
+  @Test
+  public void batchJoinFullOuter_outputValues() {
+    execute(
+        new JoinTestCase<Integer, Long, String>() {
+
+          @Override
+          protected Dataset<String> getOutput(Dataset<Integer> left, Dataset<Long> right) {
+            return FullJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Optional<Long> r, Collector<String> c) ->
+                        c.collect(l.orElse(null) + "+" + r.orElse(null)))
+                .outputValues();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<String> getUnorderedOutput() {
+            return Arrays.asList(
+                "0+null", "2+12", "2+12", "4+14", "1+11", "1+11", "3+13", "3+13", "null+15");
+          }
+        });
+  }
+
+  @Test
+  public void batchJoinLeftOuter() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return LeftJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Integer l, Optional<Long> r, Collector<String> c) ->
+                        c.collect(l + "+" + r.orElse(null)))
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, "0+null"),
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"));
+          }
+        });
+  }
+
+  @Test
+  public void batchJoinRightOuter() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return RightJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Long r, Collector<String> c) ->
+                        c.collect(l.orElse(null) + "+" + r))
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"),
+                KV.of(5, "null+15"));
+          }
+        });
+  }
+
+  @Test
+  public void batchJoinInner() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return Join.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Integer l, Long r, Collector<String> c) -> {
+                      c.collect(l + "+" + r);
+                    })
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"));
+          }
+        });
+  }
+
+  @Test
+  public void windowJoinFullOuter() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            @SuppressWarnings("unchecked")
+            final WindowFn<Object, BoundedWindow> evenOddWindowFn =
+                (WindowFn) new EvenOddWindowFn();
+            return FullJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Optional<Long> r, Collector<String> c) -> {
+                      c.collect(l.orElse(null) + "+" + r.orElse(null));
+                    })
+                .windowBy(evenOddWindowFn)
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1, 5, 6);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L, 16L, 17L, 18L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, "0+null"),
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(6, "6+16"),
+                KV.of(8, "null+18"),
+                KV.of(1, "1+null"),
+                KV.of(1, "1+null"),
+                KV.of(1, "null+11"),
+                KV.of(3, "3+null"),
+                KV.of(3, "3+null"),
+                KV.of(3, "null+13"),
+                KV.of(5, "5+null"),
+                KV.of(5, "null+15"),
+                KV.of(7, "null+17"));
+          }
+        });
+  }
+
+  @Test
+  public void windowJoinLeftOuter() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            @SuppressWarnings("unchecked")
+            final WindowFn<Object, BoundedWindow> evenOddWindowFn =
+                (WindowFn) new EvenOddWindowFn();
+            return LeftJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Integer l, Optional<Long> r, Collector<String> c) -> {
+                      c.collect(l + "+" + r.orElse(null));
+                    })
+                .windowBy(evenOddWindowFn)
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1, 5, 6);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L, 16L, 17L, 18L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, "0+null"),
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(6, "6+16"),
+                KV.of(1, "1+null"),
+                KV.of(1, "1+null"),
+                KV.of(3, "3+null"),
+                KV.of(3, "3+null"),
+                KV.of(5, "5+null"));
+          }
+        });
+  }
+
+  @Test
+  public void windowJoinRightOuter() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            @SuppressWarnings("unchecked")
+            final WindowFn<Object, BoundedWindow> evenOddWindowFn =
+                (WindowFn) new EvenOddWindowFn();
+            return RightJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Long r, Collector<String> c) -> {
+                      c.collect(l.orElse(null) + "+" + r);
+                    })
+                .windowBy(evenOddWindowFn)
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(1, 2, 3, 0, 4, 3, 2, 1, 5, 6);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L, 15L, 16L, 17L, 18L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(4, "4+14"),
+                KV.of(6, "6+16"),
+                KV.of(8, "null+18"),
+                KV.of(1, "null+11"),
+                KV.of(3, "null+13"),
+                KV.of(5, "null+15"),
+                KV.of(7, "null+17"));
+          }
+        });
+  }
+
+  // ~ all of the inputs fall into the same session window (on the same key)
+  // ~ we expect the result to reflect this fact
+  // ~ note: no early triggering
+  @Test
+  public void joinOnSessionWindowingNoEarlyTriggering() {
+    execute(
+        new JoinTestCase<KV<String, Long>, KV<String, Long>, KV<String, String>>() {
+
+          @Override
+          protected List<KV<String, Long>> getLeftInput() {
+            return Arrays.asList(KV.of("fi", 1L), KV.of("fa", 2L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<String, Long>> getLeftInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.strings(), TypeDescriptors.longs());
+          }
+
+          @Override
+          protected List<KV<String, Long>> getRightInput() {
+            return Arrays.asList(KV.of("ha", 1L), KV.of("ho", 4L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<String, Long>> getRightInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.strings(), TypeDescriptors.longs());
+          }
+
+          @Override
+          protected Dataset<KV<String, String>> getOutput(
+              Dataset<KV<String, Long>> left, Dataset<KV<String, Long>> right) {
+
+            left =
+                AssignEventTime.named("assign-event-time-left")
+                    .of(left)
+                    .using(KV::getValue)
+                    .output();
+            right =
+                AssignEventTime.named("assign-event-time-right")
+                    .of(right)
+                    .using(KV::getValue)
+                    .output();
+
+            final Dataset<KV<String, KV<String, String>>> joined =
+                Join.of(left, right)
+                    .by(p -> "", p -> "")
+                    .using(
+                        (KV<String, Long> l, KV<String, Long> r, Collector<KV<String, String>> c) ->
+                            c.collect(KV.of(l.getKey(), r.getKey())))
+                    .windowBy(Sessions.withGapDuration(org.joda.time.Duration.millis(10)))
+                    .triggeredBy(AfterWatermark.pastEndOfWindow())
+                    .discardingFiredPanes()
+                    .withAllowedLateness(Duration.ZERO)
+                    .output();
+
+            return MapElements.of(joined).using(KV::getValue).output();
+          }
+
+          @Override
+          public List<KV<String, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of("fi", "ha"), KV.of("fi", "ho"), KV.of("fa", "ha"), KV.of("fa", "ho"));
+          }
+        });
+  }
+
+  //  @Ignore(
+  //      "This test is based on access to various objects through Environment which is "
+  //          + "unsupported feature. It may be possible to add this feature in future.")
+  //  @Test
+  //  public void testJoinAccumulators() {
+  //    execute(
+  //        new JoinTestCase<
+  //            KV<String, Long>, KV<String, Long>, Triple<TimeInterval, String, String>>() {
+  //
+  //          @Override
+  //          protected List<KV<String, Long>> getLeftInput() {
+  //            return Arrays.asList(KV.of("fi", 1L), KV.of("fa", 3L));
+  //          }
+  //
+  //          @Override
+  //          protected List<KV<String, Long>> getRightInput() {
+  //            return Arrays.asList(KV.of("ha", 1L), KV.of("ho", 4L));
+  //          }
+  //
+  //          @Override
+  //          protected Dataset<Triple<TimeInterval, String, String>> getOutput(
+  //              Dataset<KV<String, Long>> left, Dataset<KV<String, Long>> right) {
+  //
+  //            left = AssignEventTime.of(left).using(KV::getValue).output();
+  //            right = AssignEventTime.of(right).using(KV::getValue).output();
+  //
+  //            Dataset<KV<String, Triple<TimeInterval, String, String>>> joined =
+  //                Join.of(left, right)
+  //                    .by(p -> "", p -> "")
+  //                    .using(
+  //                        (KV<String, Long> l,
+  //                            KV<String, Long> r,
+  //                            Collector<Triple<TimeInterval, String, String>> c) -> {
+  //                          TimeInterval window = (TimeInterval) c.getWindow();
+  //                          c.getCounter("cntr").increment(10);
+  //                          c.getHistogram("hist-" + l.getKey().charAt(1)).add(2345, 8);
+  //                          c.collect(Triple.of(window, l.getKey(), r.getKey()));
+  //                        })
+  //                    //                    .windowBy(Time.of(Duration.ofMillis(3)))
+  //                    .windowBy(FixedWindows.of(org.joda.time.Duration.millis(3)))
+  //                    .triggeredBy(AfterWatermark.pastEndOfWindow())
+  //                    .discardingFiredPanes()
+  //                    .output();
+  //
+  //            return MapElements.of(joined).using(KV::getValue).output();
+  //          }
+  //
+  //          @Override
+  //          public List<Triple<TimeInterval, String, String>> getUnorderedOutput() {
+  //            return Arrays.asList(
+  //                Triple.of(new TimeInterval(0, 3), "fi", "ha"),
+  //                Triple.of(new TimeInterval(3, 6), "fa", "ho"));
+  //          }
+  //
+  //          @Override
+  //          public void validateAccumulators(SnapshotProvider snapshots) {
+  //            Map<String, Long> counters = snapshots.getCounterSnapshots();
+  //            assertEquals(Long.valueOf(20L), counters.get("cntr"));
+  //
+  //            Map<String, Map<Long, Long>> histograms = snapshots.getHistogramSnapshots();
+  //            Map<Long, Long> hist = histograms.get("hist-i");
+  //            assertEquals(1, hist.size());
+  //            assertEquals(Long.valueOf(8), hist.get(2345L));
+  //
+  //            hist = histograms.get("hist-a");
+  //            assertEquals(1, hist.size());
+  //            assertEquals(Long.valueOf(8), hist.get(2345L));
+  //          }
+  //        });
+  //  }
+
+  @Test
+  public void batchJoinFullOuterMultipleOutputsPerCollectorFunction() {
+    execute(
+        new JoinTestCase<Integer, Long, KV<Integer, String>>() {
+
+          @Override
+          protected Dataset<KV<Integer, String>> getOutput(
+              Dataset<Integer> left, Dataset<Long> right) {
+            return FullJoin.of(left, right)
+                .by(e -> e, e -> (int) (e % 10))
+                .using(
+                    (Optional<Integer> l, Optional<Long> r, Collector<String> c) -> {
+                      // output everything twice
+                      c.collect(l.orElse(null) + "+" + r.orElse(null));
+                      c.collect(l.orElse(null) + "+" + r.orElse(null));
+                    })
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getLeftInput() {
+            return Arrays.asList(0, 1, 2, 3);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getLeftInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected List<Long> getRightInput() {
+            return Arrays.asList(11L, 12L, 13L, 14L);
+          }
+
+          @Override
+          protected TypeDescriptor<Long> getRightInputType() {
+            return TypeDescriptors.longs();
+          }
+
+          @Override
+          public List<KV<Integer, String>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, "0+null"),
+                KV.of(0, "0+null"),
+                KV.of(1, "1+11"),
+                KV.of(1, "1+11"),
+                KV.of(2, "2+12"),
+                KV.of(2, "2+12"),
+                KV.of(3, "3+13"),
+                KV.of(3, "3+13"),
+                KV.of(4, "null+14"),
+                KV.of(4, "null+14"));
+          }
+        });
+  }
+
+  /** Base for join test cases. */
+  public abstract static class JoinTestCase<LeftT, RightT, OutputT> implements TestCase<OutputT> {
+
+    @Override
+    public Dataset<OutputT> getOutput(Pipeline pipeline) {
+      final Dataset<LeftT> left =
+          Dataset.of(
+              pipeline
+                  .apply("left-input", Create.of(getLeftInput()))
+                  .setTypeDescriptor(getLeftInputType()));
+      final Dataset<RightT> right =
+          Dataset.of(
+              pipeline
+                  .apply("right-input", Create.of(getRightInput()))
+                  .setTypeDescriptor(getRightInputType()));
+      return getOutput(left, right);
+    }
+
+    protected abstract Dataset<OutputT> getOutput(Dataset<LeftT> left, Dataset<RightT> right);
+
+    protected abstract List<LeftT> getLeftInput();
+
+    protected abstract TypeDescriptor<LeftT> getLeftInputType();
+
+    protected abstract List<RightT> getRightInput();
+
+    protected abstract TypeDescriptor<RightT> getRightInputType();
+  }
+
+  /**
+   * Elements with even numeric values are are assigned to one 'even' window. All others are
+   * assigned to window named 'win: #', where '#' is value of assigned element.
+   */
+  private static class EvenOddWindowFn extends WindowFn<KV<Integer, Number>, BoundedWindow> {
+
+    private static final NamedGlobalWindow EVEN_WIN = new NamedGlobalWindow("even");
+
+    @Override
+    public Collection<BoundedWindow> assignWindows(AssignContext c) {
+      final KV<Integer, Number> element = c.element();
+      final Number value = element.getValue();
+      if (value == null) {
+        return Collections.singleton(EVEN_WIN);
+      }
+      final NamedGlobalWindow win;
+      if (value.longValue() % 2 == 0) {
+        win = EVEN_WIN;
+      } else {
+        win = new NamedGlobalWindow("win: " + value.longValue());
+      }
+      return Collections.singleton(win);
+    }
+
+    @Override
+    public void mergeWindows(MergeContext c) {
+      // no merging
+    }
+
+    @Deprecated
+    @Override
+    public boolean isCompatible(WindowFn<?, ?> other) {
+      return other instanceof EvenOddWindowFn;
+    }
+
+    @Override
+    public Coder<BoundedWindow> windowCoder() {
+      return KryoCoder.withoutClassRegistration();
+    }
+
+    @Override
+    @Nullable
+    public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
+      return null;
+    }
+
+    @Override
+    public boolean isNonMerging() {
+      return true;
+    }
+  }
+
+  private static class NamedGlobalWindow extends BoundedWindow {
+
+    private String name;
+
+    NamedGlobalWindow(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public Instant maxTimestamp() {
+      return GlobalWindow.INSTANCE.maxTimestamp();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (other instanceof NamedGlobalWindow) {
+        return name.equals(((NamedGlobalWindow) other).name);
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return name.hashCode();
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java
new file mode 100644
index 00000000000..82c4bd81ca5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction;
+import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctionEnv;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SnapshotProvider;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Tests for operator {@code MapElements}. */
+public class MapElementsTest extends AbstractOperatorTest {
+
+  @Test
+  public void testMapElements() {
+    execute(
+        new AbstractTestCase<Integer, String>() {
+
+          @Override
+          protected Dataset<String> getOutput(Dataset<Integer> input) {
+            return MapElements.of(input)
+                .using((UnaryFunction<Integer, String>) String::valueOf)
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<String> getUnorderedOutput() {
+            return Arrays.asList("1", "2", "3", "4", "5", "6", "7");
+          }
+        });
+  }
+
+  @Test
+  public void testAccumulators() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            return MapElements.named("test")
+                .of(input)
+                .using(
+                    (UnaryFunctionEnv<Integer, Integer>)
+                        (x, context) -> {
+                          context.getHistogram("dist").add(x, 1);
+                          return x;
+                        })
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 1, 2, 2, 10, 20, 10);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 2, 3, 1, 2, 2, 10, 20, 10);
+          }
+
+          @Override
+          public void validateAccumulators(SnapshotProvider snapshots) {
+            Map<Long, Long> hists = snapshots.getHistogramSnapshots().get("dist");
+            assertEquals(5, hists.size());
+            assertEquals(Long.valueOf(2), hists.get(1L));
+            assertEquals(Long.valueOf(3), hists.get(2L));
+            assertEquals(Long.valueOf(1), hists.get(3L));
+            assertEquals(Long.valueOf(2), hists.get(10L));
+            assertEquals(Long.valueOf(1), hists.get(20L));
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java
new file mode 100644
index 00000000000..5055f73320e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java
@@ -0,0 +1,811 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.collect.Sets;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Fold;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums;
+import org.apache.beam.sdk.extensions.euphoria.core.coder.KryoCoder;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SnapshotProvider;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/** Test operator {@code ReduceByKey}. */
+public class ReduceByKeyTest extends AbstractOperatorTest {
+
+  /** Validates the output type upon a `.reduceBy` operation on global window. */
+  @Test
+  public void testReductionType0() {
+    execute(
+        new AbstractTestCase<Integer, KV<Integer, Set<Integer>>>() {
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 9);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected Dataset<KV<Integer, Set<Integer>>> getOutput(Dataset<Integer> input) {
+            return ReduceByKey.of(input)
+                .keyBy(e -> e % 2)
+                .valueBy(e -> e)
+                .reduceBy(s -> s.collect(Collectors.toSet()))
+                .windowBy(new GlobalWindows())
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          public List<KV<Integer, Set<Integer>>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(0, Sets.newHashSet(2, 4, 6)), KV.of(1, Sets.newHashSet(1, 3, 5, 7, 9)));
+          }
+        });
+  }
+
+  /** Validates the output type upon a `.reduceBy` operation on global window. */
+  @Test
+  public void testReductionType0_outputValues() {
+    execute(
+        new AbstractTestCase<Integer, Set<Integer>>() {
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 9);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected Dataset<Set<Integer>> getOutput(Dataset<Integer> input) {
+            return ReduceByKey.of(input)
+                .keyBy(e -> e % 2)
+                .valueBy(e -> e)
+                .reduceBy(s -> s.collect(Collectors.toSet()))
+                .outputValues();
+          }
+
+          @Override
+          public List<Set<Integer>> getUnorderedOutput() {
+            return Arrays.asList(Sets.newHashSet(2, 4, 6), Sets.newHashSet(1, 3, 5, 7, 9));
+          }
+        });
+  }
+
+  //  /** Validates the output type upon a `.reduceBy` operation on global window. */
+  //  @Ignore("Sorting of values is not supported yet.")
+  //  @Test
+  //  public void testReductionType0WithSortedValues() {
+  //    execute(
+  //        new AbstractTestCase<Integer, List<KV<Integer, List<Integer>>>>() {
+  //
+  //          @Override
+  //          protected List<Integer> getInput() {
+  //            return Arrays.asList(9, 8, 7, 6, 5, 4, 3, 2, 1);
+  //          }
+  //
+  //          @Override
+  //          protected TypeDescriptor<Integer> getInputType() {
+  //            return TypeDescriptors.integers();
+  //          }
+  //
+  //          @Override
+  //          protected Dataset<List<KV<Integer, List<Integer>>>> getOutput(Dataset<Integer>
+  // input) {
+  //            Dataset<KV<Integer, List<Integer>>> reducedByWindow =
+  //                ReduceByKey.of(input)
+  //                    .keyBy(e -> e % 2)
+  //                    .valueBy(e -> e)
+  //                    .reduceBy(s -> s.collect(Collectors.toList()))
+  //                    .withSortedValues(Integer::compare)
+  //                    //.windowBy(Count.of(3)) //TODO rewrite to Beam windowing
+  //                    .output();
+  //
+  //            return ReduceWindow.of(reducedByWindow)
+  //                .reduceBy(s -> s.collect(Collectors.toList()))
+  //                .withSortedValues(
+  //                    (l, r) -> {
+  //                      int cmp = l.getKey().compareTo(r.getKey());
+  //                      if (cmp == 0) {
+  //                        int firstLeft = l.getValue().get(0);
+  //                        int firstRight = r.getValue().get(0);
+  //                        cmp = Integer.compare(firstLeft, firstRight);
+  //                      }
+  //                      return cmp;
+  //                    })
+  //                .windowBy(new GlobalWindows())
+  //                .triggeredBy(AfterWatermark.pastEndOfWindow())
+  //                .discardingFiredPanes()
+  //                .output();
+  //          }
+  //
+  //          @Override
+  //          public void validate(List<List<KV<Integer, List<Integer>>>> outputs)
+  //              throws AssertionError {
+  //
+  //            assertEquals(1, outputs.size());
+  //            assertEquals(
+  //                Lists.newArrayList(
+  //                    KV.of(0, Lists.newArrayList(2)),
+  //                    KV.of(0, Lists.newArrayList(4, 6, 8)),
+  //                    KV.of(1, Lists.newArrayList(1, 3)),
+  //                    KV.of(1, Lists.newArrayList(5, 7, 9))),
+  //                outputs.get(0));
+  //          }
+  //        });
+  //  }
+
+  @Test
+  public void testEventTime() {
+    execute(
+        new AbstractTestCase<KV<Integer, Long>, KV<Integer, Long>>() {
+
+          @Override
+          protected Dataset<KV<Integer, Long>> getOutput(Dataset<KV<Integer, Long>> input) {
+            input = AssignEventTime.of(input).using(KV::getValue).output();
+            return ReduceByKey.of(input)
+                .keyBy(KV::getKey)
+                .valueBy(e -> 1L)
+                .combineBy(Sums.ofLongs())
+                .windowBy(FixedWindows.of(org.joda.time.Duration.standardSeconds(1)))
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          protected List<KV<Integer, Long>> getInput() {
+            return Arrays.asList(
+                KV.of(1, 300L),
+                KV.of(2, 600L),
+                KV.of(3, 900L),
+                KV.of(2, 1300L),
+                KV.of(3, 1600L),
+                KV.of(1, 1900L),
+                KV.of(3, 2300L),
+                KV.of(2, 2600L),
+                KV.of(1, 2900L),
+                KV.of(2, 3300L),
+                KV.of(2, 300L),
+                KV.of(4, 600L),
+                KV.of(3, 900L),
+                KV.of(4, 1300L),
+                KV.of(2, 1600L),
+                KV.of(3, 1900L),
+                KV.of(4, 2300L),
+                KV.of(1, 2600L),
+                KV.of(3, 2900L),
+                KV.of(4, 3300L),
+                KV.of(3, 3600L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<Integer, Long>> getInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.longs());
+          }
+
+          @Override
+          public List<KV<Integer, Long>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(2, 2L),
+                KV.of(4, 1L), // first window
+                KV.of(2, 2L),
+                KV.of(4, 1L), // second window
+                KV.of(2, 1L),
+                KV.of(4, 1L), // third window
+                KV.of(2, 1L),
+                KV.of(4, 1L), // fourth window
+                KV.of(1, 1L),
+                KV.of(3, 2L), // first window
+                KV.of(1, 1L),
+                KV.of(3, 2L), // second window
+                KV.of(1, 2L),
+                KV.of(3, 2L), // third window
+                KV.of(3, 1L)); // fourth window
+          }
+        });
+  }
+
+  @Test
+  public void testReduceWithWindowing() {
+    execute(
+        new AbstractTestCase<Integer, KV<Integer, Long>>() {
+
+          @Override
+          protected Dataset<KV<Integer, Long>> getOutput(Dataset<Integer> input) {
+            @SuppressWarnings("unchecked")
+            final WindowFn<Object, CountWindow> windowFn = (WindowFn) new TestWindowFn();
+            return ReduceByKey.of(input)
+                .keyBy(e -> e % 3)
+                .valueBy(e -> 1L)
+                .combineBy(Sums.ofLongs())
+                .windowBy(windowFn)
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            final List<Integer> input = new ArrayList<>();
+            // first window, keys 1, 2, 0
+            input.addAll(Arrays.asList(1, 2, 3));
+            // second window, keys 1, 2, 0, 1
+            input.addAll(Arrays.asList(4, 5, 6, 7));
+            // third window, kes 2, 0, 1
+            input.addAll(Arrays.asList(8, 9, 10));
+            // second window, keys 2, 0, 1
+            input.addAll(Arrays.asList(5, 6, 7));
+            // third window, keys 2, 0, 1, 2
+            input.addAll(Arrays.asList(8, 9, 10, 11));
+            // fourth window, keys 0, 1, 2, 0
+            input.addAll(Arrays.asList(12, 13, 14, 15));
+            return input;
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<KV<Integer, Long>> getUnorderedOutput() {
+            final List<KV<Integer, Long>> output = new ArrayList<>();
+            // first window
+            output.addAll(Arrays.asList(KV.of(0, 1L), KV.of(2, 1L)));
+            // second window
+            output.addAll(Arrays.asList(KV.of(0, 2L), KV.of(2, 2L)));
+            // third window
+            output.addAll(Arrays.asList(KV.of(0, 2L), KV.of(2, 3L)));
+            // fourth window
+            output.addAll(Arrays.asList(KV.of(0, 2L), KV.of(2, 1L)));
+            // first window
+            output.add(KV.of(1, 1L));
+            // second window
+            output.add(KV.of(1, 3L));
+            // third window
+            output.add(KV.of(1, 2L));
+            // fourth window
+            output.add(KV.of(1, 1L));
+            return output;
+          }
+        });
+  }
+
+  @Test
+  public void testReduceWithoutWindowing() {
+    execute(
+        new AbstractTestCase<String, KV<String, Long>>() {
+
+          @Override
+          protected List<String> getInput() {
+            String[] words =
+                "one two three four one two three four one two three one two one".split(" ");
+            return Arrays.asList(words);
+          }
+
+          @Override
+          protected TypeDescriptor<String> getInputType() {
+            return TypeDescriptors.strings();
+          }
+
+          @Override
+          public List<KV<String, Long>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of("one", 5L), KV.of("two", 4L), KV.of("three", 3L), KV.of("four", 2L));
+          }
+
+          @Override
+          protected Dataset<KV<String, Long>> getOutput(Dataset<String> input) {
+            return ReduceByKey.of(input)
+                .keyBy(e -> e, TypeDescriptor.of(String.class))
+                .valueBy(e -> 1L, TypeDescriptor.of(Long.class))
+                .combineBy(Sums.ofLongs(), TypeDescriptor.of(Long.class))
+                .output();
+          }
+        });
+  }
+
+  @Ignore("Sorting of values is not supported yet.")
+  @Test
+  public void testReduceSorted() {
+    execute(
+        new AbstractTestCase<KV<String, Long>, KV<String, List<Long>>>() {
+
+          @Override
+          protected List<KV<String, Long>> getInput() {
+            return Arrays.asList(
+                KV.of("one", 3L),
+                KV.of("one", 2L),
+                KV.of("one", 1L),
+                KV.of("two", 3L),
+                KV.of("two", 2L),
+                KV.of("two", 1L),
+                KV.of("three", 3L),
+                KV.of("three", 2L),
+                KV.of("three", 1L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<String, Long>> getInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.strings(), TypeDescriptors.longs());
+          }
+
+          @Override
+          public List<KV<String, List<Long>>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of("one", Arrays.asList(1L, 2L, 3L)),
+                KV.of("two", Arrays.asList(1L, 2L, 3L)),
+                KV.of("three", Arrays.asList(1L, 2L, 3L)));
+          }
+
+          @Override
+          protected Dataset<KV<String, List<Long>>> getOutput(Dataset<KV<String, Long>> input) {
+            return ReduceByKey.of(input)
+                .keyBy(KV::getKey)
+                .valueBy(KV::getValue)
+                .reduceBy(
+                    (Stream<Long> values, Collector<List<Long>> coll) ->
+                        coll.collect(values.collect(Collectors.toList())))
+                .withSortedValues(Long::compareTo)
+                .output();
+          }
+        });
+  }
+
+  @Ignore("Test adaption to Beam windowing failed so far.")
+  @Test
+  public void testMergingAndTriggering() {
+    execute(
+        new AbstractTestCase<KV<String, Long>, KV<String, Long>>() {
+
+          @Override
+          protected List<KV<String, Long>> getInput() {
+            return Arrays.asList(
+                KV.of("a", 20L),
+                KV.of("c", 3_000L),
+                KV.of("b", 10L),
+                KV.of("b", 100L),
+                KV.of("a", 4_000L),
+                KV.of("c", 300L),
+                KV.of("b", 1_000L),
+                KV.of("b", 50_000L),
+                KV.of("a", 100_000L),
+                KV.of("a", 800L),
+                KV.of("a", 80L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<String, Long>> getInputType() {
+            return TypeDescriptors.kvs(TypeDescriptors.strings(), TypeDescriptors.longs());
+          }
+
+          @Override
+          protected Dataset<KV<String, Long>> getOutput(Dataset<KV<String, Long>> input) {
+            return ReduceByKey.of(input)
+                .keyBy(KV::getKey)
+                .valueBy(KV::getValue)
+                .combineBy(Sums.ofLongs())
+                .windowBy(new MergingByBucketSizeWindowFn<>(3))
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @SuppressWarnings("unchecked")
+          @Override
+          public List<KV<String, Long>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of("a", 880L),
+                KV.of("a", 104_020L),
+                KV.of("b", 1_110L),
+                KV.of("b", 50_000L),
+                KV.of("c", 3_300L));
+          }
+        });
+  }
+
+  //  // ----------------------------------------------------------------------------
+  //  @Ignore("Test depends on yet unsupported functionality (access to window from Collector). ")
+  //  @Test
+  //  public void testSessionWindowing() {
+  //    execute(
+  //        new AbstractTestCase<KV<String, Integer>, Triple<TimeInterval, Integer, Set<String>>>
+  // () {
+  //
+  //          @Override
+  //          protected List<KV<String, Integer>> getInput() {
+  //            return Arrays.asList(
+  //                KV.of("1-one", 1),
+  //                KV.of("2-one", 2),
+  //                KV.of("1-two", 4),
+  //                KV.of("1-three", 8),
+  //                KV.of("1-four", 10),
+  //                KV.of("2-two", 10),
+  //                KV.of("1-five", 18),
+  //                KV.of("2-three", 20),
+  //                KV.of("1-six", 22));
+  //          }
+  //
+  //          @Override
+  //          protected Dataset<Triple<TimeInterval, Integer, Set<String>>> getOutput(
+  //              Dataset<KV<String, Integer>> input) {
+  //            input = AssignEventTime.of(input).using(KV::getValue).output();
+  //            Dataset<KV<Integer, Set<String>>> reduced =
+  //                ReduceByKey.of(input)
+  //                    .keyBy(e -> e.getKey().charAt(0) - '0')
+  //                    .valueBy(KV::getKey)
+  //                    .reduceBy(s -> s.collect(Collectors.toSet()))
+  //                    .windowBy(FixedWindows.of(org.joda.time.Duration.millis(5)))
+  //                    .triggeredBy(AfterWatermark.pastEndOfWindow())
+  //                    .discardingFiredPanes()
+  //                    .output();
+  //
+  //            return FlatMap.of(reduced)
+  //                .using(
+  //                    (UnaryFunctor<
+  //                            KV<Integer, Set<String>>, Triple<TimeInterval, Integer,
+  // Set<String>>>)
+  //                        (elem, context) ->
+  //                            context.collect(
+  //                                Triple.of(
+  //                                    (TimeInterval) context.getWindow(),
+  //                                    elem.getKey(),
+  //                                    elem.getValue())))
+  //                .output();
+  //          }
+  //
+  //          @Override
+  //          public List<Triple<TimeInterval, Integer, Set<String>>> getUnorderedOutput() {
+  //            return Arrays.asList(
+  //                Triple.of(
+  //                    new TimeInterval(1, 15),
+  //                    1,
+  //                    Sets.newHashSet("1-four", "1-one", "1-three", "1-two")),
+  //                Triple.of(new TimeInterval(10, 15), 2, Sets.newHashSet("2-two")),
+  //                Triple.of(new TimeInterval(18, 27), 1, Sets.newHashSet("1-five", "1-six")),
+  //                Triple.of(new TimeInterval(2, 7), 2, Sets.newHashSet("2-one")),
+  //                Triple.of(new TimeInterval(20, 25), 2, Sets.newHashSet("2-three")));
+  //          }
+  //        });
+  //  }
+
+  @Test
+  public void testReduceByKeyWithWrongHashCodeImpl() {
+    execute(
+        new AbstractTestCase<KV<Word, Long>, KV<Word, Long>>() {
+
+          @Override
+          protected Dataset<KV<Word, Long>> getOutput(Dataset<KV<Word, Long>> input) {
+            input = AssignEventTime.of(input).using(KV::getValue).output();
+            return ReduceByKey.of(input)
+                .keyBy(KV::getKey)
+                .valueBy(e -> 1L)
+                .combineBy(Sums.ofLongs())
+                .windowBy(FixedWindows.of(org.joda.time.Duration.standardSeconds(1)))
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          protected List<KV<Word, Long>> getInput() {
+            return Arrays.asList(
+                KV.of(new Word("euphoria"), 300L),
+                KV.of(new Word("euphoria"), 600L),
+                KV.of(new Word("spark"), 900L),
+                KV.of(new Word("euphoria"), 1300L),
+                KV.of(new Word("flink"), 1600L),
+                KV.of(new Word("spark"), 1900L));
+          }
+
+          @Override
+          protected TypeDescriptor<KV<Word, Long>> getInputType() {
+            return TypeDescriptors.kvs(new TypeDescriptor<Word>() {}, TypeDescriptors.longs());
+          }
+
+          @Override
+          public List<KV<Word, Long>> getUnorderedOutput() {
+            return Arrays.asList(
+                KV.of(new Word("euphoria"), 2L),
+                KV.of(new Word("spark"), 1L), // first window
+                KV.of(new Word("euphoria"), 1L),
+                KV.of(new Word("spark"), 1L), // second window
+                KV.of(new Word("flink"), 1L));
+          }
+        });
+  }
+
+  @Test
+  public void testAccumulators() {
+    execute(
+        new AbstractTestCase<Integer, KV<Integer, Integer>>() {
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          protected Dataset<KV<Integer, Integer>> getOutput(Dataset<Integer> input) {
+            return ReduceByKey.named("test")
+                .of(input)
+                .keyBy(e -> e % 2)
+                .valueBy(e -> e)
+                .reduceBy(
+                    Fold.of(
+                        0,
+                        (Integer a, Integer b, Collector<Integer> ctx) -> {
+                          if (b % 2 == 0) {
+                            ctx.getCounter("evens").increment();
+                          } else {
+                            ctx.getCounter("odds").increment();
+                          }
+                          ctx.collect(a + b);
+                        }))
+                .windowBy(new GlobalWindows())
+                .triggeredBy(AfterWatermark.pastEndOfWindow())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @SuppressWarnings("unchecked")
+          @Override
+          public List<KV<Integer, Integer>> getUnorderedOutput() {
+            return Arrays.asList(KV.of(1, 9), KV.of(0, 6));
+          }
+
+          @Override
+          public void validateAccumulators(SnapshotProvider snapshots) {
+            Map<String, Long> counters = snapshots.getCounterSnapshots();
+            assertEquals(Long.valueOf(2), counters.get("evens"));
+            assertEquals(Long.valueOf(3), counters.get("odds"));
+          }
+        });
+  }
+
+  private static class TestWindowFn extends WindowFn<Number, CountWindow> {
+
+    @Override
+    public Collection<CountWindow> assignWindows(AssignContext c) throws Exception {
+      Number element = c.element();
+      return Collections.singleton(new CountWindow(element.longValue() / 4));
+    }
+
+    @Override
+    public void mergeWindows(MergeContext c) throws Exception {}
+
+    @Override
+    public boolean isNonMerging() {
+      return true;
+    }
+
+    @Deprecated
+    @Override
+    public boolean isCompatible(WindowFn<?, ?> other) {
+      return false;
+    }
+
+    @Override
+    public Coder<CountWindow> windowCoder() {
+      return KryoCoder.withoutClassRegistration();
+    }
+
+    @Override
+    @Nullable
+    public WindowMappingFn<CountWindow> getDefaultWindowMappingFn() {
+      return null;
+    }
+  }
+
+  // ~ ------------------------------------------------------------------------------
+
+  private static class CountWindow extends BoundedWindow {
+
+    private long value;
+
+    private CountWindow(long value) {
+      this.value = value;
+    }
+
+    @Override
+    public Instant maxTimestamp() {
+      return GlobalWindow.INSTANCE.maxTimestamp();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (other instanceof CountWindow) {
+        return value == (((CountWindow) other).value);
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return Long.hashCode(value);
+    }
+  }
+
+  private static class UniqueWindow extends BoundedWindow {
+
+    private static final AtomicInteger idCounter = new AtomicInteger();
+    private final int id;
+
+    private UniqueWindow() {
+      this.id = idCounter.getAndIncrement();
+    }
+
+    @Override
+    public Instant maxTimestamp() {
+      return GlobalWindow.INSTANCE.maxTimestamp();
+    }
+
+    @Override
+    public int hashCode() {
+      return Integer.hashCode(id);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj instanceof UniqueWindow && this.id == ((UniqueWindow) obj).id;
+    }
+
+    @Override
+    public String toString() {
+      return "UniqueWindow{id=" + id + "}";
+    }
+  }
+
+  private static class MergingByBucketSizeWindowFn<T> extends WindowFn<T, UniqueWindow> {
+
+    private final int bucketSize;
+
+    private MergingByBucketSizeWindowFn(int bucketSize) {
+      this.bucketSize = bucketSize;
+    }
+
+    @Override
+    public Collection<UniqueWindow> assignWindows(AssignContext c) throws Exception {
+      return Collections.singleton(new UniqueWindow());
+    }
+
+    @Override
+    public void mergeWindows(MergeContext c) throws Exception {
+
+      //       merge windows up to bucket size
+      Collection<UniqueWindow> windows = c.windows();
+      List<UniqueWindow> merges = new ArrayList<>();
+      for (UniqueWindow w : windows) {
+
+        merges.add(w);
+
+        if (merges.size() == bucketSize) { // time to merge
+          c.merge(merges, w);
+          merges.clear();
+        }
+      }
+
+      if (merges.size() > 1) {
+        c.merge(merges, merges.get(merges.size() - 1));
+      }
+    }
+
+    @Deprecated
+    @Override
+    public boolean isCompatible(WindowFn<?, ?> other) {
+      return other instanceof MergingByBucketSizeWindowFn
+          && this.bucketSize == ((MergingByBucketSizeWindowFn) other).bucketSize;
+    }
+
+    @Override
+    public Coder<UniqueWindow> windowCoder() {
+      return KryoCoder.withoutClassRegistration();
+    }
+
+    @Override
+    @Nullable
+    public WindowMappingFn<UniqueWindow> getDefaultWindowMappingFn() {
+      return null;
+    }
+  }
+
+  /** String with invalid hash code implementation returning constant. */
+  public static class Word implements Serializable {
+
+    private final String str;
+
+    Word(String str) {
+      this.str = str;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+
+      if (!(o instanceof Word)) {
+        return false;
+      }
+
+      Word word = (Word) o;
+
+      return !(str != null ? !str.equals(word.str) : word.str != null);
+    }
+
+    @Override
+    public int hashCode() {
+      return 42;
+    }
+
+    @Override
+    public String toString() {
+      return str;
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceWindowTest.java
new file mode 100644
index 00000000000..92fb61c6605
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceWindowTest.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test operator {@code ReduceByKey}. */
+public class ReduceWindowTest extends AbstractOperatorTest {
+
+  @Test
+  public void testReduceWithWindowing() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            Dataset<Integer> withEventTime =
+                AssignEventTime.of(input).using(i -> 1000L * i).output();
+
+            return ReduceWindow.of(withEventTime)
+                .combineBy(Sums.ofInts())
+                .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Collections.singletonList(55);
+          }
+        });
+  }
+
+  @Test
+  public void testReduceWithAttachedWindowing() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            Dataset<Integer> withEventTime =
+                AssignEventTime.of(input).using(i -> 1000L * i).output();
+
+            Dataset<Integer> first =
+                ReduceWindow.named("first-reduce")
+                    .of(withEventTime)
+                    .combineBy(Sums.ofInts())
+                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                    .triggeredBy(DefaultTrigger.of())
+                    .discardingFiredPanes()
+                    .output();
+
+            return ReduceWindow.named("second-reduce").of(first).combineBy(Sums.ofInts()).output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Collections.singletonList(55);
+          }
+        });
+  }
+
+  @Test
+  public void testReduceWithAttachedWindowingMoreWindows() {
+    execute(
+        new AbstractTestCase<Integer, Integer>() {
+          @Override
+          protected Dataset<Integer> getOutput(Dataset<Integer> input) {
+            Dataset<Integer> withEventTime =
+                AssignEventTime.of(input).using(i -> 1000L * i).output();
+
+            Dataset<Integer> first =
+                ReduceWindow.named("first-reduce")
+                    .of(withEventTime)
+                    .combineBy(Sums.ofInts())
+                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardSeconds(5)))
+                    .triggeredBy(DefaultTrigger.of())
+                    .discardingFiredPanes()
+                    .output();
+
+            return ReduceWindow.named("second-reduce").of(first).combineBy(Sums.ofInts()).output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 100);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return TypeDescriptors.integers();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(10, 35, 10, 100);
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java
new file mode 100644
index 00000000000..3a24e514dc8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.SumByKey;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.junit.Test;
+
+/** Test operator {@code SumByKey}. */
+public class SumByKeyTest extends AbstractOperatorTest {
+
+  @Test
+  public void testSumByKey() {
+    execute(
+        new AbstractTestCase<Integer, KV<Integer, Long>>() {
+          @Override
+          protected Dataset<KV<Integer, Long>> getOutput(Dataset<Integer> input) {
+            final Dataset<Integer> inputWithTime = AssignEventTime.of(input).using(i -> 0).output();
+            return SumByKey.of(inputWithTime)
+                .keyBy(e -> e % 2)
+                .valueBy(e -> (long) e)
+                .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .output();
+          }
+
+          @Override
+          protected List<Integer> getInput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9);
+          }
+
+          @Override
+          protected TypeDescriptor<Integer> getInputType() {
+            return null;
+          }
+
+          @Override
+          public List<KV<Integer, Long>> getUnorderedOutput() {
+            return Arrays.asList(KV.of(0, 20L), KV.of(1, 25L));
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/TopPerKeyTest.java
new file mode 100644
index 00000000000..fa4a47233d8
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/TopPerKeyTest.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import static java.util.Arrays.asList;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.TopPerKey;
+import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.Duration;
+import org.junit.Test;
+
+/** Correctness tests of {@link TopPerKey}. */
+public class TopPerKeyTest extends AbstractOperatorTest {
+
+  @Test
+  public void testAllInOneWindow() {
+    execute(
+        new AbstractTestCase<Item, Triple<String, String, Integer>>() {
+
+          @Override
+          protected Dataset<Triple<String, String, Integer>> getOutput(Dataset<Item> input) {
+            final Dataset<Item> timestampedElements =
+                AssignEventTime.of(input).using(Item::getTimestamp).output();
+            return TopPerKey.of(timestampedElements)
+                .keyBy(Item::getKey)
+                .valueBy(Item::getValue)
+                .scoreBy(Item::getScore)
+                .windowBy(FixedWindows.of(Duration.millis(10)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          public List<Triple<String, String, Integer>> getUnorderedOutput() {
+            return asList(
+                Triple.of("one", "one-999", 999),
+                Triple.of("two", "two", 10),
+                Triple.of("three", "3-three", 2));
+          }
+
+          @Override
+          protected List<Item> getInput() {
+            return asList(
+                new Item("one", "one-ZZZ-1", 1, 0L),
+                new Item("one", "one-ZZZ-2", 2, 1L),
+                new Item("one", "one-3", 3, 2L),
+                new Item("one", "one-999", 999, 3L),
+                new Item("two", "two", 10, 4L),
+                new Item("three", "1-three", 1, 5L),
+                new Item("three", "2-three", 0, 6L),
+                new Item("one", "one-XXX-100", 100, 7L),
+                new Item("three", "3-three", 2, 8L));
+          }
+
+          @Override
+          protected TypeDescriptor<Item> getInputType() {
+            return new TypeDescriptor<Item>() {};
+          }
+        });
+  }
+
+  @Test
+  public void testTwoWindows() {
+    execute(
+        new AbstractTestCase<Item, Triple<String, String, Integer>>() {
+
+          @Override
+          protected Dataset<Triple<String, String, Integer>> getOutput(Dataset<Item> input) {
+            final Dataset<Item> timestampedElements =
+                AssignEventTime.of(input).using(Item::getTimestamp).output();
+            return TopPerKey.of(timestampedElements)
+                .keyBy(Item::getKey)
+                .valueBy(Item::getValue)
+                .scoreBy(Item::getScore)
+                .windowBy(FixedWindows.of(Duration.millis(10)))
+                .triggeredBy(DefaultTrigger.of())
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO)
+                .output();
+          }
+
+          @Override
+          public List<Triple<String, String, Integer>> getUnorderedOutput() {
+            return asList(
+                // first window
+                Triple.of("one", "one-999", 999),
+                Triple.of("two", "two", 10),
+                Triple.of("three", "3-three", 2),
+                // second window
+                Triple.of("one", "one-XXX-100", 100),
+                Triple.of("three", "2-three", 0));
+          }
+
+          @Override
+          protected List<Item> getInput() {
+            return asList(
+                new Item("one", "one-ZZZ-1", 1, 14L),
+                new Item("one", "one-ZZZ-2", 2, 1L),
+                new Item("one", "one-3", 3, 13L),
+                new Item("one", "one-999", 999, 3L),
+                new Item("two", "two", 10, 4L),
+                new Item("three", "1-three", 1, 5L),
+                new Item("three", "2-three", 0, 16L),
+                new Item("one", "one-XXX-100", 100, 12L),
+                new Item("three", "3-three", 2, 8L));
+          }
+
+          @Override
+          protected TypeDescriptor<Item> getInputType() {
+            return new TypeDescriptor<Item>() {};
+          }
+        });
+  }
+
+  static final class Item implements Serializable {
+
+    private final String key, value;
+    private final int score;
+    private final long timestamp;
+
+    Item(String key, String value, int score, long timestamp) {
+      this.key = key;
+      this.value = value;
+      this.score = score;
+      this.timestamp = timestamp;
+    }
+
+    String getKey() {
+      return key;
+    }
+
+    String getValue() {
+      return value;
+    }
+
+    int getScore() {
+      return score;
+    }
+
+    long getTimestamp() {
+      return timestamp;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      Item item = (Item) o;
+      return score == item.score
+          && timestamp == item.timestamp
+          && Objects.equals(key, item.key)
+          && Objects.equals(value, item.value);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(key, value, score, timestamp);
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/UnionTest.java
new file mode 100644
index 00000000000..d47c6e42e7e
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/UnionTest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+
+/** Test for operator {@code Union}. */
+public class UnionTest extends AbstractOperatorTest {
+
+  private static Dataset<Integer> createDataset(Pipeline pipeline, Integer... data) {
+    return Dataset.of(
+        pipeline
+            .apply("create-" + UUID.randomUUID(), Create.of(Arrays.asList(data)))
+            .setTypeDescriptor(TypeDescriptors.integers()));
+  }
+
+  @Test
+  public void testUnion() {
+    execute(
+        new TestCase<Integer>() {
+
+          @Override
+          public Dataset<Integer> getOutput(Pipeline pipeline) {
+            final Dataset<Integer> first = createDataset(pipeline, 1, 2, 3, 4, 5, 6);
+            final Dataset<Integer> second = createDataset(pipeline, 7, 8, 9, 10, 11, 12);
+            return Union.of(first, second).output();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12);
+          }
+        });
+  }
+
+  @Test
+  public void testUnion_threeDataSets() {
+    execute(
+        new TestCase<Integer>() {
+
+          @Override
+          public Dataset<Integer> getOutput(Pipeline pipeline) {
+            final Dataset<Integer> first = createDataset(pipeline, 1, 2, 3, 4, 5, 6);
+            final Dataset<Integer> second = createDataset(pipeline, 7, 8, 9, 10, 11, 12);
+            final Dataset<Integer> third = createDataset(pipeline, 13, 14, 15, 16, 17, 18);
+            return Union.of(first, second, third).output();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18);
+          }
+        });
+  }
+
+  @Test
+  public void testUnion_fiveDataSets() {
+    execute(
+        new TestCase<Integer>() {
+
+          @Override
+          public Dataset<Integer> getOutput(Pipeline pipeline) {
+            final Dataset<Integer> first = createDataset(pipeline, 1, 2, 3);
+            final Dataset<Integer> second = createDataset(pipeline, 4, 5, 6);
+            final Dataset<Integer> third = createDataset(pipeline, 7, 8, 9);
+            final Dataset<Integer> fourth = createDataset(pipeline, 10, 11, 12);
+            final Dataset<Integer> fifth = createDataset(pipeline, 13, 14, 15);
+            return Union.of(first, second, third, fourth, fifth).output();
+          }
+
+          @Override
+          public List<Integer> getUnorderedOutput() {
+            return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15);
+          }
+        });
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java
new file mode 100644
index 00000000000..81b983d6326
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java
@@ -0,0 +1,488 @@
+///*
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//package org.apache.beam.sdk.extensions.euphoria.core.testkit;
+//
+//import static org.junit.Assert.assertEquals;
+//
+//import java.time.Instant;
+//import java.util.Arrays;
+//import java.util.List;
+//import java.util.Objects;
+//import java.util.concurrent.atomic.AtomicBoolean;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums;
+//import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple;
+//import org.apache.beam.sdk.extensions.euphoria.core.testkit.junit.AbstractOperatorTest;
+//import org.apache.beam.sdk.extensions.euphoria.core.testkit.junit.Processing;
+//import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+//import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+//import org.apache.beam.sdk.values.KV;
+//import org.junit.Test;
+//
+///** Tests capabilities of {@link Windowing}. */
+//@Processing(Processing.Type.ALL)
+//public class WindowingTest extends AbstractOperatorTest {
+//
+//  static final AtomicBoolean ON_CLEAR_VALIDATED = new AtomicBoolean(false);
+//
+//  @Test
+//  public void consecutiveWindowingTest_ReduceByKey() {
+//    execute(
+//        new AbstractTestCase<Triple<Instant, Type, String>, Triple<Instant, Type, Long>>() {
+//
+//          @Override
+//          protected Dataset<Triple<Instant, Type, Long>> getOutput(
+//              Dataset<Triple<Instant, Type, String>> input) {
+//
+//            input = AssignEventTime.of(input).using(t -> t.getFirst().toEpochMilli()).output();
+//            Dataset<ComparableKV<Type, String>> distinct =
+//                Distinct.of(input)
+//                    .mapped(t -> new ComparableKV<>(t.getSecond(), t.getThird()))
+//                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+//                    .triggeredBy(DefaultTrigger.of())
+//                    .discardingFiredPanes()
+//                    .output();
+//
+//            Dataset<KV<Type, Long>> reduced =
+//                ReduceByKey.of(distinct)
+//                    .keyBy(ComparableKV::getFirst)
+//                    .valueBy(p -> 1L)
+//                    .combineBy(Sums.ofLongs())
+//                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+//                    .triggeredBy(DefaultTrigger.of())
+//                    .discardingFiredPanes()
+//                    .output();
+//
+//            // extract window end timestamp
+//            return FlatMap.of(reduced)
+//                .using(
+//                    (KV<Type, Long> p, Collector<Triple<Instant, Type, Long>> ctx) -> {
+//                      long windowEnd = ((TimeInterval) ctx.getWindow()).getEndMillis();
+//                      ctx.collect(
+//                          Triple.of(Instant.ofEpochMilli(windowEnd), p.getKey(), p.getValue()));
+//                    })
+//                .output();
+//          }
+//
+//          @Override
+//          protected List<Triple<Instant, Type, String>> getInput() {
+//            return Arrays.asList(
+//                // first window
+//                Triple.of(Instant.parse("2016-12-19T10:10:00.000Z"), Type.FRUIT, "banana"),
+//                Triple.of(Instant.parse("2016-12-19T10:20:00.000Z"), Type.FRUIT, "banana"),
+//                Triple.of(Instant.parse("2016-12-19T10:25:00.000Z"), Type.FRUIT, "orange"),
+//                Triple.of(Instant.parse("2016-12-19T10:35:00.000Z"), Type.FRUIT, "apple"),
+//                Triple.of(Instant.parse("2016-12-19T10:40:00.000Z"), Type.VEGETABLE, "carrot"),
+//                Triple.of(Instant.parse("2016-12-19T10:45:00.000Z"), Type.VEGETABLE, "cucumber"),
+//                Triple.of(Instant.parse("2016-12-19T10:45:00.000Z"), Type.VEGETABLE, "cucumber"),
+//                Triple.of(Instant.parse("2016-12-19T10:50:00.000Z"), Type.VEGETABLE, "apple"),
+//
+//                // second window
+//                Triple.of(Instant.parse("2016-12-19T11:15:00.000Z"), Type.FRUIT, "banana"),
+//                Triple.of(Instant.parse("2016-12-19T11:15:00.000Z"), Type.FRUIT, "orange"),
+//                Triple.of(Instant.parse("2016-12-19T11:20:00.000Z"), Type.VEGETABLE, "carrot"),
+//                Triple.of(Instant.parse("2016-12-19T11:25:00.000Z"), Type.VEGETABLE, "carrot"));
+//          }
+//
+//          @Override
+//          public List<Triple<Instant, Type, Long>> getUnorderedOutput() {
+//            return Arrays.asList(
+//                Triple.of(Instant.parse("2016-12-19T11:00:00.000Z"), Type.FRUIT, 3L),
+//                Triple.of(Instant.parse("2016-12-19T11:00:00.000Z"), Type.VEGETABLE, 3L),
+//                Triple.of(Instant.parse("2016-12-19T12:00:00.000Z"), Type.FRUIT, 2L),
+//                Triple.of(Instant.parse("2016-12-19T12:00:00.000Z"), Type.VEGETABLE, 1L));
+//          }
+//        });
+//  }
+//
+//  @Test
+//  public void consecutiveWindowingTest_ReduceStateByKey() {
+//    execute(
+//        new AbstractTestCase<Triple<Instant, Type, String>, Triple<Instant, Type, Long>>() {
+//
+//          @Override
+//          protected Dataset<Triple<Instant, Type, Long>> getOutput(
+//              Dataset<Triple<Instant, Type, String>> input) {
+//            // distinct implemented using raw ReduceStateByKey
+//            input = AssignEventTime.of(input).using(t -> t.getFirst().toEpochMilli()).output();
+//            Dataset<KV<ComparableKV<Type, String>, Object>> keyValues =
+//                ReduceStateByKey.of(input)
+//                    .keyBy(t -> new ComparableKV<>(t.getSecond(), t.getThird()))
+//                    .valueBy(t -> null)
+//                    .stateFactory(DistinctState::new)
+//                    .mergeStatesBy((t, os) -> {})
+//                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+//                    .triggeredBy(DefaultTrigger.of())
+//                    .discardingFiredPanes()
+//                    .output();
+//
+//            Dataset<ComparableKV<Type, String>> distinct =
+//                MapElements.of(keyValues).using(KV::getKey).output();
+//
+//            Dataset<KV<Type, Long>> reduced =
+//                ReduceByKey.of(distinct)
+//                    .keyBy(ComparableKV::getFirst)
+//                    .valueBy(p -> 1L)
+//                    .combineBy(Sums.ofLongs())
+//                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+//                    .triggeredBy(DefaultTrigger.of())
+//                    .discardingFiredPanes()
+//                    .output();
+//
+//            // extract window timestamp
+//            return FlatMap.of(reduced)
+//                .using(
+//                    (KV<Type, Long> p, Collector<Triple<Instant, Type, Long>> ctx) -> {
+//                      long windowEnd = ((TimeInterval) ctx.getWindow()).getEndMillis();
+//                      ctx.collect(
+//                          Triple.of(Instant.ofEpochMilli(windowEnd), p.getKey(), p.getValue()));
+//                    })
+//                .output();
+//          }
+//
+//          @Override
+//          protected List<Triple<Instant, Type, String>> getInput() {
+//            return Arrays.asList(
+//                // first window
+//                Triple.of(Instant.parse("2016-12-19T10:10:00.000Z"), Type.FRUIT, "banana"),
+//                Triple.of(Instant.parse("2016-12-19T10:20:00.000Z"), Type.FRUIT, "banana"),
+//                Triple.of(Instant.parse("2016-12-19T10:25:00.000Z"), Type.FRUIT, "orange"),
+//                Triple.of(Instant.parse("2016-12-19T10:35:00.000Z"), Type.FRUIT, "apple"),
+//                Triple.of(Instant.parse("2016-12-19T10:40:00.000Z"), Type.VEGETABLE, "carrot"),
+//                Triple.of(Instant.parse("2016-12-19T10:45:00.000Z"), Type.VEGETABLE, "cucumber"),
+//                Triple.of(Instant.parse("2016-12-19T10:45:00.000Z"), Type.VEGETABLE, "cucumber"),
+//                Triple.of(Instant.parse("2016-12-19T10:50:00.000Z"), Type.VEGETABLE, "apple"),
+//
+//                // second window
+//                Triple.of(Instant.parse("2016-12-19T11:15:00.000Z"), Type.FRUIT, "banana"),
+//                Triple.of(Instant.parse("2016-12-19T11:15:00.000Z"), Type.FRUIT, "orange"),
+//                Triple.of(Instant.parse("2016-12-19T11:20:00.000Z"), Type.VEGETABLE, "carrot"),
+//                Triple.of(Instant.parse("2016-12-19T11:25:00.000Z"), Type.VEGETABLE, "carrot"));
+//          }
+//
+//          @SuppressWarnings("unchecked")
+//          @Override
+//          public List<Triple<Instant, Type, Long>> getUnorderedOutput() {
+//            return Arrays.asList(
+//                Triple.of(Instant.parse("2016-12-19T11:00:00.000Z"), Type.FRUIT, 3L),
+//                Triple.of(Instant.parse("2016-12-19T11:00:00.000Z"), Type.VEGETABLE, 3L),
+//                Triple.of(Instant.parse("2016-12-19T12:00:00.000Z"), Type.FRUIT, 2L),
+//                Triple.of(Instant.parse("2016-12-19T12:00:00.000Z"), Type.VEGETABLE, 1L));
+//          }
+//        });
+//  }
+//
+//  /**
+//   * Validates a trigger's #onClear method operates in the right context of merged windows.
+//   *
+//   * <p>A trigger's lifecycle is guaranteed only on stream processing; batch processing has more
+//   * freedom and doesn't necessarily invoke the trigger#onClear method.
+//   */
+//  @Processing(Processing.Type.UNBOUNDED)
+//  @Test
+//  public void testSessionWindowingTriggerStateConsistency() {
+//    ON_CLEAR_VALIDATED.set(false);
+//    execute(
+//        new AbstractTestCase<KV<Instant, String>, Triple<Instant, Instant, Integer>>(3) {
+//          @SuppressWarnings("unchecked")
+//          @Override
+//          protected Dataset<Triple<Instant, Instant, Integer>> getOutput(
+//              Dataset<KV<Instant, String>> input) {
+//            /*CSession windowing =
+//            new CSession(Duration.ofMinutes(5)) {
+//              @Override
+//              public TriggerResult onElement(
+//                  long time, TimeInterval window, TriggerContext ctx) {
+//                ValueStorage<Integer> str = ctx.getValueStorage(CSession.TR_STATE);
+//                str.set(str.get() + 1);
+//                return super.onElement(time, window, ctx);
+//              }
+//
+//              @Override
+//              public void onMerge(TimeInterval window, TriggerContext.TriggerMergeContext ctx) {
+//                ctx.mergeStoredState(CSession.TR_STATE);
+//                super.onMerge(window, ctx);
+//              }
+//
+//              @Override
+//              public TriggerResult onTimer(long time, TimeInterval window, TriggerContext ctx) {
+//                assertTrState(window, ctx);
+//                return super.onTimer(time, window, ctx);
+//              }
+//
+//              @Override
+//              public void onClear(TimeInterval window, TriggerContext ctx) {
+//                // ~ 7 minutes is the size of the final target window
+//                if (window.getDurationMillis() == Duration.ofMinutes(7).toMillis()) {
+//                  assertTrState(window, ctx);
+//                  if (!ON_CLEAR_VALIDATED.compareAndSet(false, true)) {
+//                    fail("!ON_CLEAR_VALIDATED!");
+//                  }
+//                }
+//                ctx.getValueStorage(TR_STATE).clear();
+//                super.onClear(window, ctx);
+//              }
+//
+//              private void assertTrState(TimeInterval window, TriggerContext ctx) {
+//                ValueStorage<Integer> str = ctx.getValueStorage(CSession.TR_STATE);
+//                assertEquals(3, str.get().intValue());
+//              }
+//            };
+//            */
+//
+//            input = AssignEventTime.of(input).using(t -> t.getKey().toEpochMilli()).output();
+//            Dataset<KV<String, Integer>> keyValues =
+//                ReduceByKey.of(input)
+//                    .keyBy(e -> "")
+//                    .valueBy(e -> 1)
+//                    .combineBy(Sums.ofInts())
+//                    .output();
+//
+//            // extract window timestamp
+//            return FlatMap.of(keyValues)
+//                .using(
+//                    (KV<String, Integer> in, Collector<Triple<Instant, Instant, Integer>> out) -> {
+//                      long windowBegin = ((TimeInterval) out.getWindow()).getStartMillis();
+//                      long windowEnd = ((TimeInterval) out.getWindow()).getEndMillis();
+//                      out.collect(
+//                          Triple.of(
+//                              Instant.ofEpochMilli(windowBegin),
+//                              Instant.ofEpochMilli(windowEnd),
+//                              in.getValue()));
+//                    })
+//                .output();
+//          }
+//
+//          @Override
+//          protected List<KV<Instant, String>> getInput() {
+//            return Arrays.asList(
+//                KV.of(Instant.parse("2016-12-19T10:10:00.000Z"), "foo"),
+//                KV.of(Instant.parse("2016-12-19T10:11:00.000Z"), "foo"),
+//                KV.of(Instant.parse("2016-12-19T10:12:00.000Z"), "foo"));
+//          }
+//
+//          @Override
+//          public List<Triple<Instant, Instant, Integer>> getUnorderedOutput() {
+//            return Arrays.asList(
+//                Triple.of(
+//                    Instant.parse("2016-12-19T10:10:00.000Z"),
+//                    Instant.parse("2016-12-19T10:17:00.000Z"),
+//                    3));
+//          }
+//        });
+//    assertEquals(true, ON_CLEAR_VALIDATED.get());
+//  }
+//
+//  @Test
+//  public void testTimeWindowingElementsAtBoundaries() {
+//    execute(
+//        new AbstractTestCase<Integer, KV<TimeInterval, Integer>>() {
+//
+//          @Override
+//          protected Dataset<KV<TimeInterval, Integer>> getOutput(Dataset<Integer> input) {
+//            // interpret each input element as time and just count number of
+//            // elements inside each window
+//            Dataset<Integer> timed = AssignEventTime.of(input).using(e -> e * 1000L).output();
+//            Dataset<Integer> counts =
+//                ReduceWindow.of(timed)
+//                    .valueBy(e -> 1)
+//                    .combineBy(Sums.ofInts())
+//                    .windowBy(FixedWindows.of(org.joda.time.Duration.standardHours(1)))
+//                    .triggeredBy(DefaultTrigger.of())
+//                    .discardingFiredPanes()
+//                    .output();
+//
+//            return Datasets.extractWindow(counts);
+//          }
+//
+//          @Override
+//          protected List<Integer> getInput() {
+//            return Arrays.asList(0, 1, 3599, 3599, 3600, 3600, 7199);
+//          }
+//
+//          @SuppressWarnings("unchecked")
+//          @Override
+//          public List<KV<TimeInterval, Integer>> getUnorderedOutput() {
+//            return Arrays.asList(
+//                KV.of(new TimeInterval(0, 3600000), 4),
+//                KV.of(new TimeInterval(3600000, 7200000), 3));
+//          }
+//        });
+//  }
+//
+//  /** Just simple enum to be used during testing. */
+//  public enum Type {
+//    FRUIT,
+//    VEGETABLE
+//  }
+//
+//  private static class DistinctState implements State<Object, Object> {
+//
+//    private final ValueStorage<Object> storage;
+//
+//    DistinctState(StateContext context, Collector<Object> collector) {
+//      this.storage =
+//          context
+//              .getStorageProvider()
+//              .getValueStorage(ValueStorageDescriptor.of("element", Object.class, null));
+//    }
+//
+//    @Override
+//    public void add(Object element) {
+//      storage.set(element);
+//    }
+//
+//    @Override
+//    public void flush(Collector<Object> context) {
+//      context.collect(storage.get());
+//    }
+//
+//    @Override
+//    public void close() {
+//      storage.clear();
+//    }
+//  }
+//
+//  /**
+//   * KV of items where both items implement {@link Comparable}.
+//   *
+//   * @param <T0> first item type
+//   * @param <T1> second item type
+//   */
+//  public static class ComparableKV<T0 extends Comparable<T0>, T1 extends Comparable<T1>>
+//      implements Comparable<ComparableKV<T0, T1>> {
+//
+//    private final T0 first;
+//    private final T1 second;
+//
+//    ComparableKV(T0 first, T1 second) {
+//      this.first = first;
+//      this.second = second;
+//    }
+//
+//    public T0 getFirst() {
+//      return first;
+//    }
+//
+//    public T1 getSecond() {
+//      return second;
+//    }
+//
+//    @Override
+//    public boolean equals(Object o) {
+//      if (o instanceof ComparableKV) {
+//        ComparableKV<?, ?> that = (ComparableKV<?, ?>) o;
+//        return Objects.equals(this.first, that.first) && Objects.equals(this.second, that.second);
+//      }
+//      return false;
+//    }
+//
+//    @Override
+//    public int hashCode() {
+//      return Objects.hash(first, second);
+//    }
+//
+//    @Override
+//    public int compareTo(ComparableKV<T0, T1> o) {
+//      int result = getFirst().compareTo(o.getFirst());
+//      if (result == 0) {
+//        result = getSecond().compareTo(o.getSecond());
+//      }
+//      return result;
+//    }
+//  }
+//
+//  /*
+//  static class CSession<T> implements MergingWindowing<T, TimeInterval>, Trigger<TimeInterval> {
+//
+//    static final ValueStorageDescriptor<Integer> TR_STATE =
+//        ValueStorageDescriptor.of("quux", Integer.class, 0, (x, y) -> x + y);
+//
+//    private final TimeTrigger trigger = new TimeTrigger();
+//    private final Session<T> wrap;
+//
+//    CSession(Duration gap) {
+//      wrap = Session.of(gap);
+//    }
+//
+//    @Override
+//    public Collection<KV<Collection<TimeInterval>, TimeInterval>> mergeWindows(
+//        Collection<TimeInterval> actives) {
+//      return wrap.mergeWindows(actives);
+//    }
+//
+//    @Override
+//    public Iterable<TimeInterval> assignWindowsToElement(WindowedElement<?, T> el) {
+//      return wrap.assignWindowsToElement(el);
+//    }
+//
+//    @Override
+//    public Trigger<TimeInterval> getTrigger() {
+//      return this;
+//    }
+//
+//    @Override
+//    public TriggerResult onElement(long time, TimeInterval window, TriggerContext ctx) {
+//      return trigger.onElement(time, window, ctx);
+//    }
+//
+//    @Override
+//    public TriggerResult onTimer(long time, TimeInterval window, TriggerContext ctx) {
+//      return trigger.onTimer(time, window, ctx);
+//    }
+//
+//    @Override
+//    public void onClear(TimeInterval window, TriggerContext ctx) {
+//      trigger.onClear(window, ctx);
+//    }
+//
+//    @Override
+//    public void onMerge(TimeInterval window, TriggerContext.TriggerMergeContext ctx) {
+//      trigger.onMerge(window, ctx);
+//    }
+//
+//    @Override
+//    public boolean equals(Object obj) {
+//      if (obj instanceof CSession) {
+//        CSession other = (CSession) obj;
+//        return other.wrap.equals(wrap);
+//      }
+//      return false;
+//    }
+//
+//    @Override
+//    public int hashCode() {
+//      return wrap.hashCode();
+//    }
+//  }
+//  */
+//}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/LongCounter.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/LongCounter.java
new file mode 100644
index 00000000000..9a68bbf46af
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/LongCounter.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+
+final class LongCounter implements Counter, Snapshotable<Long> {
+
+  private AtomicLong value = new AtomicLong();
+
+  LongCounter() {}
+
+  @Override
+  public void increment(long value) {
+    this.value.addAndGet(value);
+  }
+
+  @Override
+  public void increment() {
+    increment(1);
+  }
+
+  @Override
+  public Long getSnapshot() {
+    return value.get();
+  }
+
+  @Override
+  public String toString() {
+    return "LongCounter{" + "value=" + value + '}';
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/LongHistogram.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/LongHistogram.java
new file mode 100644
index 00000000000..05b1e47ea29
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/LongHistogram.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+
+final class LongHistogram implements Histogram, Snapshotable<Map<Long, Long>> {
+
+  final Map<Long, Long> buckets = new ConcurrentHashMap<>();
+
+  LongHistogram() {}
+
+  @Override
+  public void add(long value, long times) {
+    buckets.compute(value, (key, count) -> count == null ? times : (count + times));
+  }
+
+  @Override
+  public void add(long value) {
+    add(value, 1);
+  }
+
+  @Override
+  public Map<Long, Long> getSnapshot() {
+    return new HashMap<>(buckets);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/NanosecondTimer.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/NanosecondTimer.java
new file mode 100644
index 00000000000..3b289c86ba5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/NanosecondTimer.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
+
+import com.google.common.collect.Maps;
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+
+final class NanosecondTimer implements Timer, Snapshotable<Map<Duration, Long>> {
+
+  private final LongHistogram hist = new LongHistogram();
+
+  NanosecondTimer() {}
+
+  @Override
+  public void add(long duration, TimeUnit unit) {
+    hist.add(unit.toNanos(duration), 1);
+  }
+
+  @Override
+  public void add(Duration duration) {
+    add(duration.toNanos(), TimeUnit.NANOSECONDS);
+  }
+
+  @Override
+  public Map<Duration, Long> getSnapshot() {
+    Map<Duration, Long> m = Maps.newHashMapWithExpectedSize(hist.buckets.size());
+    hist.buckets.forEach((key, count) -> m.put(Duration.ofNanos(key), count));
+    return m;
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/SingleJvmAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/SingleJvmAccumulatorProvider.java
new file mode 100644
index 00000000000..ae44dc004f5
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/SingleJvmAccumulatorProvider.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
+
+import java.io.ObjectStreamException;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Accumulator;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
+
+/**
+ * An accumulator provider gathering accumulators in-memory.
+ *
+ * <p>Safe for use for unit testing purposes.
+ */
+public class SingleJvmAccumulatorProvider implements AccumulatorProvider {
+
+  private static final SingleJvmAccumulatorProvider INSTANCE = new SingleJvmAccumulatorProvider();
+  private final ConcurrentMap<String, Accumulator> accs = new ConcurrentHashMap<>();
+
+  private SingleJvmAccumulatorProvider() {}
+
+  @SuppressWarnings("unchecked")
+  private static <T> T assertType(String name, Class<T> expectedType, Accumulator actualAcc) {
+    if (actualAcc.getClass() != expectedType) {
+      // ~ provide a nice message (that's why we don't simply use `expectedType.cast(..)`)
+      throw new IllegalStateException(
+          "Ambiguously named accumulators! Got "
+              + actualAcc.getClass()
+              + " for "
+              + name
+              + " but expected "
+              + expectedType
+              + "!");
+    }
+    return (T) actualAcc;
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return assertType(name, LongCounter.class, accs.computeIfAbsent(name, s -> new LongCounter()));
+  }
+
+  @Override
+  public Counter getCounter(String namespace, String name) {
+    return getCounter(name);
+  }
+
+  @Override
+  public Histogram getHistogram(String name) {
+    return assertType(
+        name, LongHistogram.class, accs.computeIfAbsent(name, s -> new LongHistogram()));
+  }
+
+  @Override
+  public Histogram getHistogram(String namespace, String name) {
+    return getHistogram(name);
+  }
+
+  @Override
+  public Timer getTimer(String name) {
+    return assertType(
+        name, NanosecondTimer.class, accs.computeIfAbsent(name, s -> new NanosecondTimer()));
+  }
+
+  void clear() {
+    accs.clear();
+  }
+
+  <V, T extends Snapshotable<V>> Map<String, V> getSnapshots(Class<T> type) {
+    HashMap<String, V> m = new HashMap<>();
+    accs.forEach(
+        (name, accumulator) -> {
+          if (type.isAssignableFrom(accumulator.getClass())) {
+            @SuppressWarnings("unchecked")
+            T acc = (T) accumulator;
+            m.put(name, acc.getSnapshot());
+          }
+        });
+    return m;
+  }
+
+  // ~ -----------------------------------------------------------------------
+
+  /** Accumulator provider factory. */
+  public static final class Factory implements AccumulatorProvider.Factory, SnapshotProvider {
+
+    private static final Factory INSTANCE = new Factory();
+
+    private Factory() {}
+
+    /**
+     * Before running another test, clear method must be called to delete metrics snapshots
+     *
+     * @return SingleJvmAccumulatorFactory
+     */
+    public static Factory get() {
+      return INSTANCE;
+    }
+
+    public void clear() {
+      providerInstance().clear();
+    }
+
+    @Override
+    public Map<String, Long> getCounterSnapshots() {
+      return providerInstance().getSnapshots(LongCounter.class);
+    }
+
+    @Override
+    public Map<String, Map<Long, Long>> getHistogramSnapshots() {
+      return providerInstance().getSnapshots(LongHistogram.class);
+    }
+
+    @Override
+    public Map<String, Map<Duration, Long>> getTimerSnapshots() {
+      return providerInstance().getSnapshots(NanosecondTimer.class);
+    }
+
+    @Override
+    public AccumulatorProvider create() {
+      return providerInstance();
+    }
+
+    private SingleJvmAccumulatorProvider providerInstance() {
+      return SingleJvmAccumulatorProvider.INSTANCE;
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+      return INSTANCE;
+    }
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/SnapshotProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/SnapshotProvider.java
new file mode 100644
index 00000000000..6bb09448e43
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/SnapshotProvider.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
+
+import java.time.Duration;
+import java.util.Map;
+
+/** Provider of snapshots. */
+public interface SnapshotProvider {
+
+  Map<String, Long> getCounterSnapshots();
+
+  Map<String, Map<Long, Long>> getHistogramSnapshots();
+
+  Map<String, Map<Duration, Long>> getTimerSnapshots();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/Snapshotable.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/Snapshotable.java
new file mode 100644
index 00000000000..4cb0497850f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/Snapshotable.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
+
+interface Snapshotable<V> {
+  V getSnapshot();
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/package-info.java
new file mode 100644
index 00000000000..d87cc092e4f
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/accumulators/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Accumulators to be used in operator test suite. */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/package-info.java
new file mode 100644
index 00000000000..a93dfc377f6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Euphoria operators test suite. */
+package org.apache.beam.sdk.extensions.euphoria.core.testkit;
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BeamMetricsTranslationTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BeamMetricsTranslationTest.java
new file mode 100644
index 00000000000..db9f4aa9c81
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BeamMetricsTranslationTest.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import static org.apache.beam.sdk.metrics.MetricResultsMatchers.metricsResult;
+import static org.junit.Assert.assertThat;
+
+import java.util.stream.Stream;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey;
+import org.apache.beam.sdk.extensions.euphoria.core.coder.KryoCoder;
+import org.apache.beam.sdk.metrics.DistributionResult;
+import org.apache.beam.sdk.metrics.MetricNameFilter;
+import org.apache.beam.sdk.metrics.MetricQueryResults;
+import org.apache.beam.sdk.metrics.MetricsFilter;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Testing translation of accumulators to Beam {@link org.apache.beam.sdk.metrics.Metrics}. */
+public class BeamMetricsTranslationTest {
+
+  @Rule public TestPipeline testPipeline = TestPipeline.create();
+
+  @Before
+  public void setup() {
+    testPipeline
+        .getCoderRegistry()
+        .registerCoderForClass(Object.class, KryoCoder.withoutClassRegistration());
+  }
+
+  /**
+   * Test metrics counters on {@link ReduceByKey} and {@link MapElements} operators Flow:
+   *
+   * <ol>
+   *   <li>step RBK increment for all keys, add to histogram its value, collect even numbers.
+   *   <li>step MapElements increment for every element, add to histogram its value, map to integer.
+   *   <li>tep test MapElements with default operator name, increment by value of its element, add
+   *       to histogram 2 times value of its element.
+   * </ol>
+   */
+  @Test
+  public void testBeamMetricsTranslation() {
+    final PCollection<Integer> input =
+        testPipeline.apply("input", Create.of(1, 2, 3, 4, 5).withType(TypeDescriptors.integers()));
+    final String counterName1 = "counter1";
+    final String operatorName1 = "count_elements_and_save_even_numbers";
+
+    final Dataset<KV<Integer, Integer>> kvInput =
+        ReduceByKey.named(operatorName1)
+            .of(Dataset.of(input))
+            .keyBy(e -> e)
+            .reduceBy(
+                (Stream<Integer> list, Collector<Integer> coll) ->
+                    list.forEach(
+                        i -> {
+                          coll.getCounter(counterName1).increment();
+                          coll.getHistogram(counterName1).add(i);
+                          if (i % 2 == 0) {
+                            coll.collect(i);
+                          }
+                        }))
+            .output();
+
+    final String counterName2 = "counter2";
+    final String operatorName2 = "map_to_integer";
+    final String operatorName3 = "map_elements";
+
+    final Dataset<Integer> mapElementsOutput =
+        MapElements.named(operatorName2)
+            .of(kvInput) // kvInput = [<2,2>, <4,4>]
+            .using(
+                (kv, context) -> {
+                  final Integer value = kv.getValue();
+                  context.getCounter(counterName2).increment();
+                  context.getHistogram(counterName2).add(value);
+                  return value;
+                })
+            .output();
+
+    final Dataset<Integer> output =
+        MapElements.named(operatorName3)
+            .of(mapElementsOutput) // mapElementsOutput = [2,4]
+            .using(
+                (value, context) -> {
+                  context.getCounter(counterName2).increment(value);
+                  context.getHistogram(counterName2).add(value, 2);
+                  return value;
+                })
+            .output();
+
+    PAssert.that(output.getPCollection()).containsInAnyOrder(2, 4);
+
+    final PipelineResult result = testPipeline.run();
+    result.waitUntilFinish();
+
+    final MetricQueryResults metricQueryResults =
+        result
+            .metrics()
+            .queryMetrics(
+                MetricsFilter.builder()
+                    .addNameFilter(MetricNameFilter.inNamespace(operatorName1))
+                    .addNameFilter(MetricNameFilter.inNamespace(operatorName2))
+                    .addNameFilter(MetricNameFilter.inNamespace(operatorName3))
+                    .build());
+
+    testStep1Metrics(metricQueryResults, counterName1, operatorName1);
+    testStep2Metrics(metricQueryResults, counterName2, operatorName2);
+    testStep3WithDefaultOperatorName(metricQueryResults, counterName2, operatorName3);
+  }
+
+  private void testStep1Metrics(MetricQueryResults metrics, String counterName1, String stepName1) {
+    assertThat(
+        metrics.getCounters(),
+        Matchers.hasItem(metricsResult(stepName1, counterName1, stepName1, 5L, true)));
+
+    assertThat(
+        metrics.getDistributions(),
+        Matchers.hasItem(
+            metricsResult(
+                stepName1, counterName1, stepName1, DistributionResult.create(15, 5, 1, 5), true)));
+  }
+
+  private void testStep2Metrics(MetricQueryResults metrics, String counterName2, String stepName2) {
+    assertThat(
+        metrics.getCounters(),
+        Matchers.hasItem(metricsResult(stepName2, counterName2, stepName2, 2L, true)));
+
+    assertThat(
+        metrics.getDistributions(),
+        Matchers.hasItem(
+            metricsResult(
+                stepName2, counterName2, stepName2, DistributionResult.create(6, 2, 2, 4), true)));
+  }
+
+  private void testStep3WithDefaultOperatorName(
+      MetricQueryResults metrics, String counterName2, String stepName3) {
+    assertThat(
+        metrics.getCounters(),
+        Matchers.hasItem(metricsResult(stepName3, counterName2, stepName3, 6L, true)));
+
+    assertThat(
+        metrics.getDistributions(),
+        Matchers.hasItem(
+            metricsResult(
+                stepName3, counterName2, stepName3, DistributionResult.create(12, 4, 2, 4), true)));
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/EuphoriaTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/EuphoriaTest.java
new file mode 100644
index 00000000000..b695275deae
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/EuphoriaTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset;
+import org.apache.beam.sdk.extensions.euphoria.core.client.lib.Euphoria;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CountByKey;
+import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements;
+import org.apache.beam.sdk.extensions.euphoria.core.coder.KryoCoder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** A group of test focused at {@link Euphoria}. */
+public class EuphoriaTest implements Serializable {
+
+  private static final String BASE_STRING =
+      "Lorem ipsum dolor sit amet, consectetur adipiscing elit. Curabitur et imperdiet nulla,"
+          + " vulputate luctus risus. In sed suscipit purus. Curabitur dui eros, eleifend sed "
+          + "dignissim eget, euismod sed lorem.";
+
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Before
+  public void setup() {
+    pipeline
+        .getCoderRegistry()
+        .registerCoderForClass(Object.class, KryoCoder.withoutClassRegistration());
+  }
+
+  @Test
+  public void basicBeamTransformTest() {
+    final List<String> words = Arrays.asList(BASE_STRING.split(" "));
+    final List<String> upperCaseWords = Arrays.asList(BASE_STRING.toUpperCase().split(" "));
+    final PCollection<String> pCollection =
+        pipeline
+            .apply("Create", Create.of(words))
+            .apply(
+                "To-UpperCase",
+                Euphoria.of(input -> MapElements.of(input).using(s -> s.toUpperCase()).output()));
+    PAssert.that(pCollection).containsInAnyOrder(upperCaseWords);
+    pipeline.run();
+  }
+
+  @Test
+  public void testChainedOperations() {
+    final String inStr = "a b c a a c x";
+    final List<String> words = Arrays.asList(inStr.split(" "));
+    final PCollection<KV<String, Long>> pCollection =
+        pipeline
+            .apply("Create", Create.of(words))
+            .apply(
+                "To-UpperCase",
+                Euphoria.of(
+                    (Dataset<String> input) -> {
+                      Dataset<String> upperCase =
+                          MapElements.of(input).using(s -> s.toUpperCase()).output();
+
+                      return CountByKey.of(upperCase).keyBy(e -> e).output();
+                    }));
+    PAssert.that(pCollection)
+        .containsInAnyOrder(
+            Arrays.asList(KV.of("A", 3L), KV.of("B", 1L), KV.of("C", 2L), KV.of("X", 1L)));
+    pipeline.run();
+  }
+
+  @Test
+  public void testBeamTransformWhenFlowIsExecuted() {
+    final List<String> words = Arrays.asList(BASE_STRING.split(" "));
+    final List<String> upperCaseWords = Arrays.asList(BASE_STRING.toUpperCase().split(" "));
+    final PCollection<String> pCollection =
+        pipeline
+            .apply("Create", Create.of(words))
+            .apply(
+                "To-UpperCase",
+                Euphoria.of(input -> MapElements.of(input).using(s -> s.toUpperCase()).output()));
+    PAssert.that(pCollection).containsInAnyOrder(upperCaseWords);
+    pipeline.run().waitUntilFinish();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java
new file mode 100644
index 00000000000..f62da777f8a
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import java.util.Map;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SingleJvmAccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SingleJvmAccumulatorProvider.Factory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link SingleJvmAccumulatorProvider}. Note that this test is placed outside of
+ * {@code testkit} package on purpouse. All the other tests in {@code testkit} are not runnable by
+ * JUnit directly.
+ */
+public class SingleJvmAccumulatorProviderTest {
+
+  private static final String TEST_COUNTER_NAME = "test-counter";
+  private static final String TEST_HISTOGRAM_NAME = "test-histogram";
+
+  private Factory accFactory = Factory.get();
+
+  @Test
+  public void testBasicAccumulatorsFunction() {
+    final AccumulatorProvider accumulators = accFactory.create();
+
+    Counter counter = accumulators.getCounter(TEST_COUNTER_NAME);
+    Assert.assertNotNull(counter);
+
+    counter.increment();
+    counter.increment(2);
+
+    Map<String, Long> counterSnapshots = accFactory.getCounterSnapshots();
+    long counterValue = counterSnapshots.get(TEST_COUNTER_NAME);
+    Assert.assertEquals(3L, counterValue);
+
+    Histogram histogram = accumulators.getHistogram(TEST_HISTOGRAM_NAME);
+    Assert.assertNotNull(histogram);
+
+    histogram.add(1);
+    histogram.add(2, 2);
+
+    Map<String, Map<Long, Long>> histogramSnapshots = accFactory.getHistogramSnapshots();
+    Map<Long, Long> histogramValue = histogramSnapshots.get(TEST_HISTOGRAM_NAME);
+
+    long numOfValuesOfOne = histogramValue.get(1L);
+    Assert.assertEquals(1L, numOfValuesOfOne);
+    long numOfValuesOfTwo = histogramValue.get(2L);
+    Assert.assertEquals(2L, numOfValuesOfTwo);
+
+    // collector.getTimer() <- not yet supported
+  }
+
+  /** need to delete all metrics from accumulator before running another test */
+  @After
+  public void cleanUp() {
+    accFactory.clear();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TestUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TestUtils.java
new file mode 100644
index 00000000000..bd4e7f9f8d6
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TestUtils.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.beam.sdk.extensions.euphoria.core.translate;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/** Collection of methods reused among tests. */
+public class TestUtils {
+
+  static PipelineOptions defaultOptions() {
+    String[] args = {"--runner=DirectRunner"};
+    return PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java
new file mode 100644
index 00000000000..63da1eb64cc
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.core.translate.collector;
+
+import java.util.Map;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter;
+import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SingleJvmAccumulatorProvider;
+import org.apache.beam.sdk.extensions.euphoria.core.testkit.accumulators.SingleJvmAccumulatorProvider.Factory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** {@link SingleValueCollector} unit tests. */
+public class SingleValueCollectorTest {
+
+  private static final String TEST_COUNTER_NAME = "test-counter";
+  private static final String TEST_HISTOGRAM_NAME = "test-histogram";
+
+  private final Factory accumulatorFactory = SingleJvmAccumulatorProvider.Factory.get();
+
+  @Test
+  public void testBasicAccumulatorsAccess() {
+
+    final AccumulatorProvider accumulators = accumulatorFactory.create();
+
+    SingleValueCollector collector = new SingleValueCollector(accumulators, "test-no_op_name");
+
+    Counter counter = collector.getCounter(TEST_COUNTER_NAME);
+    Assert.assertNotNull(counter);
+    Histogram histogram = collector.getHistogram(TEST_HISTOGRAM_NAME);
+    Assert.assertNotNull(histogram);
+
+    // collector.getTimer() <- not yet supported
+  }
+
+  @Test
+  public void testBasicAccumulatorsFunction() {
+    final AccumulatorProvider accumulators = accumulatorFactory.create();
+
+    SingleValueCollector collector = new SingleValueCollector(accumulators, "test-no_op_name");
+
+    Counter counter = collector.getCounter(TEST_COUNTER_NAME);
+    Assert.assertNotNull(counter);
+
+    counter.increment();
+    counter.increment(2);
+
+    Map<String, Long> counterSnapshots = accumulatorFactory.getCounterSnapshots();
+    long counteValue = counterSnapshots.get(TEST_COUNTER_NAME);
+    Assert.assertEquals(3L, counteValue);
+
+    Histogram histogram = collector.getHistogram(TEST_HISTOGRAM_NAME);
+    Assert.assertNotNull(histogram);
+
+    histogram.add(1);
+    histogram.add(2, 2);
+
+    Map<String, Map<Long, Long>> histogramSnapshots = accumulatorFactory.getHistogramSnapshots();
+    Map<Long, Long> histogramValue = histogramSnapshots.get(TEST_HISTOGRAM_NAME);
+
+    long numOfValuesOfOne = histogramValue.get(1L);
+    Assert.assertEquals(1L, numOfValuesOfOne);
+    long numOfValuesOfTwo = histogramValue.get(2L);
+    Assert.assertEquals(2L, numOfValuesOfTwo);
+
+    // collector.getTimer() <- not yet supported
+  }
+
+  /** need to delete all metrics from accumulator before running another test */
+  @After
+  public void cleanUp() {
+    accumulatorFactory.clear();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java
new file mode 100644
index 00000000000..80e0c4e9750
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java
@@ -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.beam.sdk.transforms.windowing;
+
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+/**
+ * Exposes window package private properties for testing
+ *
+ * @param <T> type of input element
+ */
+public class WindowDesc<T> {
+
+  public static <T> WindowDesc<T> of(Window<T> window) {
+    return new WindowDesc<>(window);
+  }
+
+  private final Window<T> window;
+
+  private WindowDesc(Window<T> window) {
+    this.window = window;
+  }
+
+  /**
+   * {@link Window#getWindowFn()}
+   *
+   * @return windowFn
+   */
+  @SuppressWarnings("unchecked")
+  public WindowFn<Object, ?> getWindowFn() {
+    return (WindowFn) window.getWindowFn();
+  }
+
+  /**
+   * {@link Window#getTrigger()}
+   *
+   * @return trigger
+   */
+  public Trigger getTrigger() {
+    return window.getTrigger();
+  }
+
+  /**
+   * {@link Window#getAccumulationMode()}
+   *
+   * @return accumulation mode
+   */
+  public WindowingStrategy.AccumulationMode getAccumulationMode() {
+    return window.getAccumulationMode();
+  }
+
+  /**
+   * {@link Window#getAllowedLateness()}
+   *
+   * @return allowed lateness
+   */
+  public Duration getAllowedLateness() {
+    return window.getAllowedLateness();
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-testing/build.gradle b/sdks/java/extensions/euphoria/euphoria-testing/build.gradle
new file mode 100644
index 00000000000..fd4423898c0
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-testing/build.gradle
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+apply plugin: org.apache.beam.gradle.BeamModulePlugin
+applyJavaNature()
+
+dependencies {
+    compile library.java.junit
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java
new file mode 100644
index 00000000000..639f43b3f7c
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.euphoria.testing;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.junit.Assert;
+
+/** Test wide utility methods for dataset assertion. */
+public class DatasetAssert {
+
+  /**
+   * Compare two datasets, no matter how they are ordered.
+   *
+   * @param <T> type of input data
+   * @param tested the tested dataset as list
+   * @param values varargs values
+   */
+  @SafeVarargs
+  public static <T> void unorderedEquals(List<T> tested, T... values) {
+    unorderedEquals(Arrays.asList(values), tested);
+  }
+
+  /**
+   * Compare two data sets, no matter how they are ordered.
+   *
+   * @param left first dataset to compare
+   * @param right second dataset to compare
+   * @param <T> type of data, that data sets contain
+   */
+  public static <T> void unorderedEquals(List<T> left, List<T> right) {
+    final Map<T, Integer> leftCounted =
+        left.stream().collect(Collectors.toMap(e -> e, e -> 1, (a, b) -> a + b));
+    final Map<T, Integer> rightCounted =
+        right.stream().collect(Collectors.toMap(e -> e, e -> 1, (a, b) -> a + b));
+    Assert.assertEquals(leftCounted, rightCounted);
+  }
+}
diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/package-info.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/package-info.java
new file mode 100644
index 00000000000..e2d4baa8689
--- /dev/null
+++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Testing related utilities. */
+package org.apache.beam.sdk.extensions.euphoria.testing;
diff --git a/settings.gradle b/settings.gradle
index 01ea889a542..518205b86c7 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -78,6 +78,11 @@ include "beam-sdks-java-container"
 project(":beam-sdks-java-container").dir = file("sdks/java/container")
 include "beam-sdks-java-core"
 project(":beam-sdks-java-core").dir = file("sdks/java/core")
+include "beam-sdks-java-extensions-euphoria-core"
+project(":beam-sdks-java-extensions-euphoria-core").dir = file("sdks/java/extensions/euphoria/euphoria-core")
+include "beam-sdks-java-extensions-euphoria-testing"
+project(":beam-sdks-java-extensions-euphoria-testing").dir = file("sdks/java/extensions/euphoria/euphoria-testing")
+
 include "beam-sdks-java-extensions-google-cloud-platform-core"
 project(":beam-sdks-java-extensions-google-cloud-platform-core").dir = file("sdks/java/extensions/google-cloud-platform-core")
 include "beam-sdks-java-extensions-json-jackson"


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 146321)
    Time Spent: 20m  (was: 10m)

> Euphoria: Full windowing support
> --------------------------------
>
>                 Key: BEAM-5393
>                 URL: https://issues.apache.org/jira/browse/BEAM-5393
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-euphoria
>            Reporter: David Moravek
>            Assignee: David Moravek
>            Priority: Minor
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> There are window properties we don't expose to user right now. We should support all properties that Window builder does.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Mime
View raw message